HBASE-12796 Clean up HTable and HBaseAdmin deprecated constructor usage (Jurriaan Mous)

This commit is contained in:
tedyu 2015-01-08 07:49:28 -08:00
parent 645fbd7d87
commit 9246af8dac
156 changed files with 1095 additions and 857 deletions

View File

@ -190,7 +190,7 @@ public class MetaTableAccessor {
// There should still be a way to use this method with an unmanaged connection. // There should still be a way to use this method with an unmanaged connection.
if (connection instanceof ClusterConnection) { if (connection instanceof ClusterConnection) {
if (((ClusterConnection) connection).isManaged()) { 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); return connection.getTable(TableName.META_TABLE_NAME);

View File

@ -41,9 +41,8 @@ import org.apache.hadoop.hbase.TableName;
import org.apache.hadoop.hbase.classification.InterfaceAudience; import org.apache.hadoop.hbase.classification.InterfaceAudience;
import org.apache.hadoop.hbase.classification.InterfaceStability; import org.apache.hadoop.hbase.classification.InterfaceStability;
import org.apache.hadoop.hbase.client.Admin; 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.HConnection; import org.apache.hadoop.hbase.client.ConnectionFactory;
import org.apache.hadoop.hbase.client.HConnectionManager;
import org.apache.hadoop.hbase.replication.ReplicationException; import org.apache.hadoop.hbase.replication.ReplicationException;
import org.apache.hadoop.hbase.replication.ReplicationFactory; import org.apache.hadoop.hbase.replication.ReplicationFactory;
import org.apache.hadoop.hbase.replication.ReplicationPeerConfig; import org.apache.hadoop.hbase.replication.ReplicationPeerConfig;
@ -91,7 +90,7 @@ public class ReplicationAdmin implements Closeable {
public static final String REPLICATIONGLOBAL = Integer public static final String REPLICATIONGLOBAL = Integer
.toString(HConstants.REPLICATION_SCOPE_GLOBAL); .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 // TODO: replication should be managed by master. All the classes except ReplicationAdmin should
// be moved to hbase-server. Resolve it in HBASE-11392. // be moved to hbase-server. Resolve it in HBASE-11392.
private final ReplicationQueuesClient replicationQueuesClient; private final ReplicationQueuesClient replicationQueuesClient;
@ -109,7 +108,7 @@ public class ReplicationAdmin implements Closeable {
throw new RuntimeException("hbase.replication isn't true, please " + throw new RuntimeException("hbase.replication isn't true, please " +
"enable it in order to use replication"); "enable it in order to use replication");
} }
this.connection = HConnectionManager.getConnection(conf); this.connection = ConnectionFactory.createConnection(conf);
ZooKeeperWatcher zkw = createZooKeeperWatcher(); ZooKeeperWatcher zkw = createZooKeeperWatcher();
try { try {
this.replicationPeers = ReplicationFactory.getReplicationPeers(zkw, conf, this.connection); 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 * Append the replicable table-cf config of the specified peer
* @param id a short that identifies the cluster * @param id a short that identifies the cluster
* @param tableCfs table-cfs config str * @param tableCfs table-cfs config str
* @throws KeeperException * @throws ReplicationException
*/ */
public void appendPeerTableCFs(String id, String tableCfs) throws ReplicationException { public void appendPeerTableCFs(String id, String tableCfs) throws ReplicationException {
appendPeerTableCFs(id, parseTableCFsFromConfig(tableCfs)); appendPeerTableCFs(id, parseTableCFsFromConfig(tableCfs));
@ -333,7 +332,7 @@ public class ReplicationAdmin implements Closeable {
* Append the replicable table-cf config of the specified peer * Append the replicable table-cf config of the specified peer
* @param id a short that identifies the cluster * @param id a short that identifies the cluster
* @param tableCfs A map from tableName to column family names * @param tableCfs A map from tableName to column family names
* @throws KeeperException * @throws ReplicationException
*/ */
public void appendPeerTableCFs(String id, Map<TableName, ? extends Collection<String>> tableCfs) public void appendPeerTableCFs(String id, Map<TableName, ? extends Collection<String>> tableCfs)
throws ReplicationException { throws ReplicationException {
@ -469,7 +468,7 @@ public class ReplicationAdmin implements Closeable {
public List<HashMap<String, String>> listReplicated() throws IOException { public List<HashMap<String, String>> listReplicated() throws IOException {
List<HashMap<String, String>> replicationColFams = new ArrayList<HashMap<String, String>>(); List<HashMap<String, String>> replicationColFams = new ArrayList<HashMap<String, String>>();
Admin admin = new HBaseAdmin(this.connection.getConfiguration()); Admin admin = connection.getAdmin();
HTableDescriptor[] tables; HTableDescriptor[] tables;
try { try {
tables = admin.listTables(); tables = admin.listTables();

View File

@ -145,7 +145,8 @@ public class TestClientNoCluster extends Configured implements Tool {
Configuration localConfig = HBaseConfiguration.create(this.conf); Configuration localConfig = HBaseConfiguration.create(this.conf);
// This override mocks up our exists/get call to throw a RegionServerStoppedException. // This override mocks up our exists/get call to throw a RegionServerStoppedException.
localConfig.set("hbase.client.connection.impl", RpcTimeoutConnection.class.getName()); 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; Throwable t = null;
LOG.info("Start"); LOG.info("Start");
try { try {
@ -161,6 +162,7 @@ public class TestClientNoCluster extends Configured implements Tool {
} finally { } finally {
table.close(); table.close();
} }
connection.close();
LOG.info("Stop"); LOG.info("Stop");
assertTrue(t != null); 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 -- // 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. // all ten of them -- and we'll get the RetriesExhaustedException exception.
localConfig.setInt(HConstants.HBASE_CLIENT_META_OPERATION_TIMEOUT, pause - 1); 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; Throwable t = null;
try { try {
// An exists call turns into a get w/ a flag. // An exists call turns into a get w/ a flag.
@ -196,6 +199,7 @@ public class TestClientNoCluster extends Configured implements Tool {
fail(); fail();
} finally { } finally {
table.close(); table.close();
connection.close();
} }
assertTrue(t != null); 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 // 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 // means we'll have to do a bunch more mocking. Tests that go against meta only should be
// good for a bit of testing. // 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); ResultScanner scanner = table.getScanner(HConstants.CATALOG_FAMILY);
try { try {
Result result = null; Result result = null;
@ -226,6 +231,7 @@ public class TestClientNoCluster extends Configured implements Tool {
} finally { } finally {
scanner.close(); scanner.close();
table.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 // 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 // means we'll have to do a bunch more mocking. Tests that go against meta only should be
// good for a bit of testing. // 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); ResultScanner scanner = table.getScanner(HConstants.CATALOG_FAMILY);
try { try {
Result result = null; Result result = null;
@ -246,6 +253,7 @@ public class TestClientNoCluster extends Configured implements Tool {
} finally { } finally {
scanner.close(); scanner.close();
table.close(); table.close();
connection.close();
} }
} }

View File

@ -125,7 +125,7 @@ public class TestBulkDeleteProtocol {
private long invokeBulkDeleteProtocol(TableName tableName, final Scan scan, final int rowBatchSize, private long invokeBulkDeleteProtocol(TableName tableName, final Scan scan, final int rowBatchSize,
final DeleteType deleteType, final Long timeStamp) throws Throwable { 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; long noOfDeletedRows = 0L;
Batch.Call<BulkDeleteService, BulkDeleteResponse> callable = Batch.Call<BulkDeleteService, BulkDeleteResponse> callable =
new Batch.Call<BulkDeleteService, BulkDeleteResponse>() { new Batch.Call<BulkDeleteService, BulkDeleteResponse>() {
@ -220,7 +220,7 @@ public class TestBulkDeleteProtocol {
htd.addFamily(new HColumnDescriptor(FAMILY1)); htd.addFamily(new HColumnDescriptor(FAMILY1));
htd.addFamily(new HColumnDescriptor(FAMILY2)); htd.addFamily(new HColumnDescriptor(FAMILY2));
TEST_UTIL.getHBaseAdmin().createTable(htd, Bytes.toBytes(0), Bytes.toBytes(120), 5); 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<Put> puts = new ArrayList<Put>(100); List<Put> puts = new ArrayList<Put>(100);
for (int j = 0; j < 100; j++) { for (int j = 0; j < 100; j++) {
Put put = new Put(Bytes.toBytes(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 hcd.setMaxVersions(10);// Just setting 10 as I am not testing with more than 10 versions here
htd.addFamily(hcd); htd.addFamily(hcd);
TEST_UTIL.getHBaseAdmin().createTable(htd, Bytes.toBytes(0), Bytes.toBytes(120), 5); 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; return ht;
} }

View File

@ -71,7 +71,7 @@ public class TestRowCountEndpoint {
// @Ignore @Test // @Ignore @Test
public void testEndpoint() throws Throwable { public void testEndpoint() throws Throwable {
Table table = new HTable(CONF, TEST_TABLE); Table table = TEST_UTIL.getConnection().getTable(TEST_TABLE);
// insert some test rows // insert some test rows
for (int i=0; i<5; i++) { for (int i=0; i<5; i++) {

View File

@ -76,7 +76,7 @@ public class TestZooKeeperScanPolicyObserver {
.setTimeToLive(1); .setTimeToLive(1);
desc.addFamily(hcd); desc.addFamily(hcd);
TEST_UTIL.getHBaseAdmin().createTable(desc); 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(); long now = EnvironmentEdgeManager.currentTime();
ZooKeeperWatcher zkw = new ZooKeeperWatcher(TEST_UTIL.getConfiguration(), "test", null); ZooKeeperWatcher zkw = new ZooKeeperWatcher(TEST_UTIL.getConfiguration(), "test", null);

View File

@ -27,6 +27,8 @@ import java.util.concurrent.atomic.AtomicLong;
import org.apache.commons.logging.Log; import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory; import org.apache.commons.logging.LogFactory;
import org.apache.hadoop.conf.Configuration; 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.HTable;
import org.apache.hadoop.hbase.client.Result; import org.apache.hadoop.hbase.client.Result;
import org.apache.hadoop.hbase.client.ResultScanner; import org.apache.hadoop.hbase.client.ResultScanner;
@ -224,7 +226,8 @@ public class IntegrationTestLazyCfLoading {
long maxRuntime = conf.getLong(timeoutKey, DEFAULT_TIMEOUT_MINUTES); long maxRuntime = conf.getLong(timeoutKey, DEFAULT_TIMEOUT_MINUTES);
long serverCount = util.getHBaseClusterInterface().getClusterStatus().getServersSize(); long serverCount = util.getHBaseClusterInterface().getClusterStatus().getServersSize();
long keysToWrite = serverCount * KEYS_TO_WRITE_PER_SERVER; 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 // Create multi-threaded writer and start it. We write multiple columns/CFs and verify
// their integrity, therefore multi-put is necessary. // their integrity, therefore multi-put is necessary.
@ -288,5 +291,6 @@ public class IntegrationTestLazyCfLoading {
Assert.assertEquals("There are write failures", 0, writer.getNumWriteFailures()); Assert.assertEquals("There are write failures", 0, writer.getNumWriteFailures());
Assert.assertTrue("Writer is not done", isWriterDone); Assert.assertTrue("Writer is not done", isWriterDone);
// Assert.fail("Boom!"); // Assert.fail("Boom!");
connection.close();
} }
} }

View File

@ -20,40 +20,35 @@ package org.apache.hadoop.hbase.chaos.actions;
import java.io.IOException; import java.io.IOException;
import java.util.Collection; import java.util.Collection;
import java.util.List;
import org.apache.commons.lang.math.RandomUtils; import org.apache.commons.lang.math.RandomUtils;
import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.hbase.HRegionLocation;
import org.apache.hadoop.hbase.ServerName; import org.apache.hadoop.hbase.ServerName;
import org.apache.hadoop.hbase.TableName; 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.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. * Action that restarts an HRegionServer holding one of the regions of the table.
*/ */
public class RestartRsHoldingTableAction extends RestartActionBaseAction { 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); super(sleepTime);
this.tableName = tableName; this.locator = locator;
} }
@Override @Override
public void perform() throws Exception { public void perform() throws Exception {
HTable table = null; LOG.info("Performing action: Restart random RS holding table " + this.locator.getName());
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;
}
Collection<ServerName> serverNames = table.getRegionLocations().values(); List<HRegionLocation> locations = locator.getAllRegionLocations();
ServerName[] nameArray = serverNames.toArray(new ServerName[serverNames.size()]); restartRs(locations.get(RandomUtils.nextInt(locations.size())).getServerName(), sleepTime);
restartRs(nameArray[RandomUtils.nextInt(nameArray.length)], sleepTime);
} }
} }

View File

@ -144,7 +144,7 @@ public class IntegrationTestImportTsv implements Configurable, Tool {
setCaching(1000); setCaching(1000);
}}; }};
try { try {
table = new HTable(getConf(), tableName); table = util.getConnection().getTable(tableName);
Iterator<Result> resultsIt = table.getScanner(scan).iterator(); Iterator<Result> resultsIt = table.getScanner(scan).iterator();
Iterator<KeyValue> expectedIt = simple_expected.iterator(); Iterator<KeyValue> expectedIt = simple_expected.iterator();
while (resultsIt.hasNext() && expectedIt.hasNext()) { while (resultsIt.hasNext() && expectedIt.hasNext()) {

View File

@ -189,7 +189,8 @@ public class IntegrationTestMTTR {
// Set up the action that will restart a region server holding a region from our table // 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. // 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. // Set up the action that will kill the region holding meta.
restartMetaAction = new RestartRsHoldingMetaAction(sleepTime); restartMetaAction = new RestartRsHoldingMetaAction(sleepTime);
@ -478,7 +479,7 @@ public class IntegrationTestMTTR {
public PutCallable(Future<?> f) throws IOException { public PutCallable(Future<?> f) throws IOException {
super(f); super(f);
this.table = new HTable(util.getConfiguration(), tableName); this.table = util.getConnection().getTable(tableName);
} }
@Override @Override
@ -504,7 +505,7 @@ public class IntegrationTestMTTR {
public ScanCallable(Future<?> f) throws IOException { public ScanCallable(Future<?> f) throws IOException {
super(f); super(f);
this.table = new HTable(util.getConfiguration(), tableName); this.table = util.getConnection().getTable(tableName);
} }
@Override @Override
@ -545,7 +546,7 @@ public class IntegrationTestMTTR {
protected boolean doAction() throws Exception { protected boolean doAction() throws Exception {
Admin admin = null; Admin admin = null;
try { try {
admin = new HBaseAdmin(util.getConfiguration()); admin = util.getHBaseAdmin();
ClusterStatus status = admin.getClusterStatus(); ClusterStatus status = admin.getClusterStatus();
return status != null; return status != null;
} finally { } finally {

View File

@ -18,18 +18,7 @@
package org.apache.hadoop.hbase.test; package org.apache.hadoop.hbase.test;
import java.io.DataInput; import com.google.common.collect.Sets;
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 org.apache.commons.cli.CommandLine; import org.apache.commons.cli.CommandLine;
import org.apache.commons.cli.GnuParser; import org.apache.commons.cli.GnuParser;
import org.apache.commons.cli.HelpFormatter; 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.HTableDescriptor;
import org.apache.hadoop.hbase.IntegrationTestBase; import org.apache.hadoop.hbase.IntegrationTestBase;
import org.apache.hadoop.hbase.IntegrationTestingUtility; 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.MasterNotRunningException;
import org.apache.hadoop.hbase.TableName; import org.apache.hadoop.hbase.TableName;
import org.apache.hadoop.hbase.client.Admin; 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.Get;
import org.apache.hadoop.hbase.client.HBaseAdmin; import org.apache.hadoop.hbase.client.HBaseAdmin;
import org.apache.hadoop.hbase.client.HConnection; import org.apache.hadoop.hbase.client.HConnection;
import org.apache.hadoop.hbase.client.HConnectionManager; 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.Put;
import org.apache.hadoop.hbase.client.Result; import org.apache.hadoop.hbase.client.Result;
import org.apache.hadoop.hbase.client.ResultScanner; import org.apache.hadoop.hbase.client.ResultScanner;
import org.apache.hadoop.hbase.client.Scan; import org.apache.hadoop.hbase.client.Scan;
import org.apache.hadoop.hbase.client.ScannerCallable; import org.apache.hadoop.hbase.client.ScannerCallable;
import org.apache.hadoop.hbase.client.Table; 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.io.ImmutableBytesWritable;
import org.apache.hadoop.hbase.mapreduce.TableMapReduceUtil; import org.apache.hadoop.hbase.mapreduce.TableMapReduceUtil;
import org.apache.hadoop.hbase.mapreduce.TableMapper; import org.apache.hadoop.hbase.mapreduce.TableMapper;
import org.apache.hadoop.hbase.mapreduce.TableRecordReaderImpl; 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.AbstractHBaseTool;
import org.apache.hadoop.hbase.util.Bytes; import org.apache.hadoop.hbase.util.Bytes;
import org.apache.hadoop.hbase.util.RegionSplitter; import org.apache.hadoop.hbase.util.RegionSplitter;
@ -97,7 +87,17 @@ import org.apache.hadoop.util.ToolRunner;
import org.junit.Test; import org.junit.Test;
import org.junit.experimental.categories.Category; 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, * This is an integration test borrowed from goraci, written by Keith Turner,
@ -340,7 +340,8 @@ public class IntegrationTestBigLinkedList extends IntegrationTestBase {
byte[] id; byte[] id;
long count = 0; long count = 0;
int i; int i;
HTable table; Table table;
Connection connection;
long numNodes; long numNodes;
long wrap; long wrap;
int width; int width;
@ -348,8 +349,8 @@ public class IntegrationTestBigLinkedList extends IntegrationTestBase {
@Override @Override
protected void setup(Context context) throws IOException, InterruptedException { protected void setup(Context context) throws IOException, InterruptedException {
id = Bytes.toBytes("Job: "+context.getJobID() + " Task: " + context.getTaskAttemptID()); id = Bytes.toBytes("Job: "+context.getJobID() + " Task: " + context.getTaskAttemptID());
Configuration conf = context.getConfiguration(); this.connection = ConnectionFactory.createConnection(context.getConfiguration());
instantiateHTable(conf); instantiateHTable();
this.width = context.getConfiguration().getInt(GENERATOR_WIDTH_KEY, WIDTH_DEFAULT); this.width = context.getConfiguration().getInt(GENERATOR_WIDTH_KEY, WIDTH_DEFAULT);
current = new byte[this.width][]; current = new byte[this.width][];
int wrapMultiplier = context.getConfiguration().getInt(GENERATOR_WRAP_KEY, WRAP_DEFAULT); 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 { protected void instantiateHTable() throws IOException {
table = new HTable(conf, getTableName(conf)); table = connection.getTable(getTableName(connection.getConfiguration()));
table.setAutoFlushTo(false); table.setAutoFlushTo(false);
table.setWriteBufferSize(4 * 1024 * 1024); table.setWriteBufferSize(4 * 1024 * 1024);
} }
@ -370,6 +371,7 @@ public class IntegrationTestBigLinkedList extends IntegrationTestBase {
@Override @Override
protected void cleanup(Context context) throws IOException ,InterruptedException { protected void cleanup(Context context) throws IOException ,InterruptedException {
table.close(); table.close();
connection.close();
} }
@Override @Override
@ -876,7 +878,8 @@ public class IntegrationTestBigLinkedList extends IntegrationTestBase {
System.exit(-1); 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 scan = new Scan();
scan.setBatch(10000); scan.setBatch(10000);
@ -906,6 +909,7 @@ public class IntegrationTestBigLinkedList extends IntegrationTestBase {
} }
scanner.close(); scanner.close();
table.close(); table.close();
connection.close();
return 0; return 0;
} }
@ -926,9 +930,10 @@ public class IntegrationTestBigLinkedList extends IntegrationTestBase {
org.apache.hadoop.hbase.client.Delete delete org.apache.hadoop.hbase.client.Delete delete
= new org.apache.hadoop.hbase.client.Delete(val); = new org.apache.hadoop.hbase.client.Delete(val);
Table table = new HTable(getConf(), getTableName(getConf())); try (Connection connection = ConnectionFactory.createConnection(getConf());
Table table = connection.getTable(getTableName(getConf()))) {
table.delete(delete); table.delete(delete);
table.close(); }
System.out.println("Delete successful"); System.out.println("Delete successful");
return 0; return 0;
@ -970,7 +975,8 @@ public class IntegrationTestBigLinkedList extends IntegrationTestBase {
byte[] startKey = isSpecificStart ? Bytes.toBytesBinary(cmd.getOptionValue('s')) : null; byte[] startKey = isSpecificStart ? Bytes.toBytesBinary(cmd.getOptionValue('s')) : null;
int logEvery = cmd.hasOption('l') ? Integer.parseInt(cmd.getOptionValue('l')) : 1; 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; long numQueries = 0;
// If isSpecificStart is set, only walk one list from that particular node. // 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 // 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(); table.close();
connection.close();
return 0; return 0;
} }

View File

@ -38,6 +38,8 @@ import org.apache.hadoop.hbase.IntegrationTestingUtility;
import org.apache.hadoop.hbase.TableName; import org.apache.hadoop.hbase.TableName;
import org.apache.hadoop.hbase.chaos.factories.MonkeyFactory; import org.apache.hadoop.hbase.chaos.factories.MonkeyFactory;
import org.apache.hadoop.hbase.client.Admin; 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.Delete;
import org.apache.hadoop.hbase.client.HBaseAdmin; import org.apache.hadoop.hbase.client.HBaseAdmin;
import org.apache.hadoop.hbase.client.HConnection; import org.apache.hadoop.hbase.client.HConnection;
@ -182,9 +184,9 @@ public class IntegrationTestBigLinkedListWithVisibility extends IntegrationTestB
} }
@Override @Override
protected void instantiateHTable(Configuration conf) throws IOException { protected void instantiateHTable() throws IOException {
for (int i = 0; i < DEFAULT_TABLES_COUNT; i++) { 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.setAutoFlushTo(true);
//table.setWriteBufferSize(4 * 1024 * 1024); //table.setWriteBufferSize(4 * 1024 * 1024);
this.tables[i] = table; this.tables[i] = table;

View File

@ -40,6 +40,9 @@ import org.apache.hadoop.hbase.HConstants;
import org.apache.hadoop.hbase.HTableDescriptor; import org.apache.hadoop.hbase.HTableDescriptor;
import org.apache.hadoop.hbase.IntegrationTestBase; import org.apache.hadoop.hbase.IntegrationTestBase;
import org.apache.hadoop.hbase.IntegrationTestingUtility; 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.testclassification.IntegrationTests;
import org.apache.hadoop.hbase.TableName; import org.apache.hadoop.hbase.TableName;
import org.apache.hadoop.hbase.client.Admin; import org.apache.hadoop.hbase.client.Admin;
@ -164,13 +167,14 @@ public void cleanUpCluster() throws Exception {
extends Mapper<NullWritable, NullWritable, NullWritable, NullWritable> extends Mapper<NullWritable, NullWritable, NullWritable, NullWritable>
{ {
protected long recordsToWrite; protected long recordsToWrite;
protected HTable table; protected Connection connection;
protected Table table;
protected Configuration conf; protected Configuration conf;
protected int numBackReferencesPerRow; protected int numBackReferencesPerRow;
protected String shortTaskId; protected String shortTaskId;
protected Random rand = new Random(); protected Random rand = new Random();
protected Counter rowsWritten, refsWritten; protected Counter rowsWritten, refsWritten;
@Override @Override
@ -179,7 +183,8 @@ public void cleanUpCluster() throws Exception {
recordsToWrite = conf.getLong(NUM_TO_WRITE_KEY, NUM_TO_WRITE_DEFAULT); recordsToWrite = conf.getLong(NUM_TO_WRITE_KEY, NUM_TO_WRITE_DEFAULT);
String tableName = conf.get(TABLE_NAME_KEY, TABLE_NAME_DEFAULT); String tableName = conf.get(TABLE_NAME_KEY, TABLE_NAME_DEFAULT);
numBackReferencesPerRow = conf.getInt(NUM_BACKREFS_KEY, NUM_BACKREFS_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.setWriteBufferSize(4*1024*1024);
table.setAutoFlushTo(false); table.setAutoFlushTo(false);
@ -198,6 +203,7 @@ public void cleanUpCluster() throws Exception {
public void cleanup(Context context) throws IOException { public void cleanup(Context context) throws IOException {
table.flushCommits(); table.flushCommits();
table.close(); table.close();
connection.close();
} }
@Override @Override

View File

@ -124,7 +124,7 @@ public class IntegrationTestSendTraceRequests extends AbstractHBaseTool {
ResultScanner rs = null; ResultScanner rs = null;
try { try {
innerScope = Trace.startSpan("Scan", Sampler.ALWAYS); innerScope = Trace.startSpan("Scan", Sampler.ALWAYS);
Table ht = new HTable(util.getConfiguration(), tableName); Table ht = util.getConnection().getTable(tableName);
Scan s = new Scan(); Scan s = new Scan();
s.setStartRow(Bytes.toBytes(rowKeyQueue.take())); s.setStartRow(Bytes.toBytes(rowKeyQueue.take()));
s.setBatch(7); s.setBatch(7);
@ -174,7 +174,7 @@ public class IntegrationTestSendTraceRequests extends AbstractHBaseTool {
Table ht = null; Table ht = null;
try { try {
ht = new HTable(util.getConfiguration(), tableName); ht = util.getConnection().getTable(tableName);
} catch (IOException e) { } catch (IOException e) {
e.printStackTrace(); e.printStackTrace();
} }
@ -234,7 +234,7 @@ public class IntegrationTestSendTraceRequests extends AbstractHBaseTool {
private LinkedBlockingQueue<Long> insertData() throws IOException, InterruptedException { private LinkedBlockingQueue<Long> insertData() throws IOException, InterruptedException {
LinkedBlockingQueue<Long> rowKeys = new LinkedBlockingQueue<Long>(25000); LinkedBlockingQueue<Long> rowKeys = new LinkedBlockingQueue<Long>(25000);
HTable ht = new HTable(util.getConfiguration(), this.tableName); Table ht = util.getConnection().getTable(this.tableName);
byte[] value = new byte[300]; byte[] value = new byte[300];
for (int x = 0; x < 5000; x++) { for (int x = 0; x < 5000; x++) {
TraceScope traceScope = Trace.startSpan("insertData", Sampler.ALWAYS); TraceScope traceScope = Trace.startSpan("insertData", Sampler.ALWAYS);

View File

@ -101,7 +101,7 @@ public class TestGzipFilter {
Response response = client.put(path, headers, value_1_gzip); Response response = client.put(path, headers, value_1_gzip);
assertEquals(response.getCode(), 200); 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 get = new Get(Bytes.toBytes(ROW_1));
get.addColumn(Bytes.toBytes(CFA), Bytes.toBytes("1")); get.addColumn(Bytes.toBytes(CFA), Bytes.toBytes("1"));
Result result = table.get(get); Result result = table.get(get);

View File

@ -137,7 +137,7 @@ public class TestScannersWithFilters {
htd.addFamily(new HColumnDescriptor(FAMILIES[0])); htd.addFamily(new HColumnDescriptor(FAMILIES[0]));
htd.addFamily(new HColumnDescriptor(FAMILIES[1])); htd.addFamily(new HColumnDescriptor(FAMILIES[1]));
admin.createTable(htd); admin.createTable(htd);
Table table = new HTable(TEST_UTIL.getConfiguration(), TABLE); Table table = TEST_UTIL.getConnection().getTable(TABLE);
// Insert first half // Insert first half
for(byte [] ROW : ROWS_ONE) { for(byte [] ROW : ROWS_ONE) {
Put p = new Put(ROW); Put p = new Put(ROW);

View File

@ -17,23 +17,6 @@
*/ */
package org.apache.hadoop.hbase.rest; 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.conf.Configuration;
import org.apache.hadoop.hbase.HBaseTestingUtility; import org.apache.hadoop.hbase.HBaseTestingUtility;
import org.apache.hadoop.hbase.HColumnDescriptor; 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.TableName;
import org.apache.hadoop.hbase.client.Admin; import org.apache.hadoop.hbase.client.Admin;
import org.apache.hadoop.hbase.client.Durability; 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.Put;
import org.apache.hadoop.hbase.client.Table; import org.apache.hadoop.hbase.client.Table;
import org.apache.hadoop.hbase.protobuf.generated.VisibilityLabelsProtos.VisibilityLabelsResponse; import org.apache.hadoop.hbase.protobuf.generated.VisibilityLabelsProtos.VisibilityLabelsResponse;
@ -69,6 +51,21 @@ import org.junit.BeforeClass;
import org.junit.Test; import org.junit.Test;
import org.junit.experimental.categories.Category; 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}) @Category({RestTests.class, MediumTests.class})
public class TestScannersWithLabels { public class TestScannersWithLabels {
private static final TableName TABLE = TableName.valueOf("TestScannersWithLabels"); private static final TableName TABLE = TableName.valueOf("TestScannersWithLabels");
@ -104,7 +101,7 @@ public class TestScannersWithLabels {
+ TOPSECRET)); + TOPSECRET));
puts.add(put); puts.add(put);
} }
try (Table table = new HTable(TEST_UTIL.getConfiguration(), tableName)) { try (Table table = TEST_UTIL.getConnection().getTable(tableName)) {
table.put(puts); table.put(puts);
} }
return puts.size(); return puts.size();

View File

@ -87,7 +87,7 @@ public class TestTableResource {
HTableDescriptor htd = new HTableDescriptor(TABLE); HTableDescriptor htd = new HTableDescriptor(TABLE);
htd.addFamily(new HColumnDescriptor(COLUMN_FAMILY)); htd.addFamily(new HColumnDescriptor(COLUMN_FAMILY));
admin.createTable(htd); 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[] k = new byte[3];
byte [][] famAndQf = KeyValue.parseColumn(Bytes.toBytes(COLUMN)); byte [][] famAndQf = KeyValue.parseColumn(Bytes.toBytes(COLUMN));
for (byte b1 = 'a'; b1 < 'z'; b1++) { for (byte b1 = 'a'; b1 < 'z'; b1++) {

View File

@ -101,7 +101,7 @@ public class TestRemoteTable {
admin.createTable(htd); admin.createTable(htd);
Table table = null; Table table = null;
try { try {
table = new HTable(TEST_UTIL.getConfiguration(), TABLE); table = TEST_UTIL.getConnection().getTable(TABLE);
Put put = new Put(ROW_1); Put put = new Put(ROW_1);
put.add(COLUMN_1, QUALIFIER_1, TS_2, VALUE_1); put.add(COLUMN_1, QUALIFIER_1, TS_2, VALUE_1);
table.put(put); table.put(put);
@ -135,7 +135,7 @@ public class TestRemoteTable {
public void testGetTableDescriptor() throws IOException { public void testGetTableDescriptor() throws IOException {
Table table = null; Table table = null;
try { try {
table = new HTable(TEST_UTIL.getConfiguration(), TABLE); table = TEST_UTIL.getConnection().getTable(TABLE);
HTableDescriptor local = table.getTableDescriptor(); HTableDescriptor local = table.getTableDescriptor();
assertEquals(remoteTable.getTableDescriptor(), local); assertEquals(remoteTable.getTableDescriptor(), local);
} finally { } finally {

View File

@ -30,7 +30,8 @@ import org.apache.hadoop.hbase.classification.InterfaceAudience;
import org.apache.hadoop.hbase.classification.InterfaceStability; import org.apache.hadoop.hbase.classification.InterfaceStability;
import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.hbase.client.Admin; 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.regionserver.HRegionServer;
import org.apache.hadoop.hbase.security.User; import org.apache.hadoop.hbase.security.User;
import org.apache.hadoop.hbase.util.JVMClusterUtil.RegionServerThread; import org.apache.hadoop.hbase.util.JVMClusterUtil.RegionServerThread;
@ -461,7 +462,8 @@ public class LocalHBaseCluster {
Configuration conf = HBaseConfiguration.create(); Configuration conf = HBaseConfiguration.create();
LocalHBaseCluster cluster = new LocalHBaseCluster(conf); LocalHBaseCluster cluster = new LocalHBaseCluster(conf);
cluster.startup(); cluster.startup();
Admin admin = new HBaseAdmin(conf); Connection connection = ConnectionFactory.createConnection(conf);
Admin admin = connection.getAdmin();
try { try {
HTableDescriptor htd = HTableDescriptor htd =
new HTableDescriptor(TableName.valueOf(cluster.getClass().getName())); new HTableDescriptor(TableName.valueOf(cluster.getClass().getName()));
@ -469,6 +471,7 @@ public class LocalHBaseCluster {
} finally { } finally {
admin.close(); admin.close();
} }
connection.close();
cluster.shutdown(); cluster.shutdown();
} }
} }

View File

@ -48,7 +48,6 @@ import org.apache.hadoop.hbase.TableName;
import org.apache.hadoop.hbase.client.HTable; import org.apache.hadoop.hbase.client.HTable;
import org.apache.hadoop.hbase.client.HTableInterface; import org.apache.hadoop.hbase.client.HTableInterface;
import org.apache.hadoop.hbase.client.HTableWrapper; 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.CoprocessorClassLoader;
import org.apache.hadoop.hbase.util.SortedCopyOnWriteSet; import org.apache.hadoop.hbase.util.SortedCopyOnWriteSet;
import org.apache.hadoop.hbase.util.VersionInfo; import org.apache.hadoop.hbase.util.VersionInfo;
@ -325,7 +324,7 @@ public abstract class CoprocessorHost<E extends CoprocessorEnvironment> {
final ClassLoader systemClassLoader = this.getClass().getClassLoader(); final ClassLoader systemClassLoader = this.getClass().getClassLoader();
for (E env : coprocessors) { for (E env : coprocessors) {
ClassLoader cl = env.getInstance().getClass().getClassLoader(); ClassLoader cl = env.getInstance().getClass().getClassLoader();
if (cl != systemClassLoader ){ if (cl != systemClassLoader){
//do not include system classloader //do not include system classloader
externalClassLoaders.add(cl); externalClassLoaders.add(cl);
} }
@ -434,7 +433,7 @@ public abstract class CoprocessorHost<E extends CoprocessorEnvironment> {
} catch (IOException e) { } catch (IOException e) {
// nothing can be done here // nothing can be done here
LOG.warn("Failed to close " + LOG.warn("Failed to close " +
Bytes.toStringBinary(table.getTableName()), e); table.getName(), e);
} }
} }
} }

View File

@ -25,8 +25,9 @@ import org.apache.commons.logging.LogFactory;
import org.apache.hadoop.hbase.classification.InterfaceAudience; import org.apache.hadoop.hbase.classification.InterfaceAudience;
import org.apache.hadoop.hbase.classification.InterfaceStability; import org.apache.hadoop.hbase.classification.InterfaceStability;
import org.apache.hadoop.fs.Path; import org.apache.hadoop.fs.Path;
import org.apache.hadoop.hbase.HBaseConfiguration;
import org.apache.hadoop.hbase.TableName; 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.HTable;
import org.apache.hadoop.hbase.util.Bytes; import org.apache.hadoop.hbase.util.Bytes;
import org.apache.hadoop.mapred.FileInputFormat; import org.apache.hadoop.mapred.FileInputFormat;
@ -58,8 +59,8 @@ public class TableInputFormat extends TableInputFormatBase implements
} }
setInputColumns(m_cols); setInputColumns(m_cols);
try { try {
setHTable( Connection connection = ConnectionFactory.createConnection(job);
new HTable(HBaseConfiguration.create(job), TableName.valueOf(tableNames[0].getName()))); setHTable((HTable) connection.getTable(TableName.valueOf(tableNames[0].getName())));
} catch (Exception e) { } catch (Exception e) {
LOG.error(StringUtils.stringifyException(e)); LOG.error(StringUtils.stringifyException(e));
} }

View File

@ -27,6 +27,7 @@ import org.apache.hadoop.hbase.classification.InterfaceStability;
import org.apache.hadoop.hbase.HConstants; import org.apache.hadoop.hbase.HConstants;
import org.apache.hadoop.hbase.client.HTable; import org.apache.hadoop.hbase.client.HTable;
import org.apache.hadoop.hbase.client.Result; 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.filter.Filter;
import org.apache.hadoop.hbase.io.ImmutableBytesWritable; import org.apache.hadoop.hbase.io.ImmutableBytesWritable;
import org.apache.hadoop.mapred.InputFormat; import org.apache.hadoop.mapred.InputFormat;
@ -158,7 +159,7 @@ implements InputFormat<ImmutableBytesWritable, Result> {
/** /**
* Allows subclasses to get the {@link HTable}. * Allows subclasses to get the {@link HTable}.
*/ */
protected HTable getHTable() { protected Table getHTable() {
return this.table; return this.table;
} }

View File

@ -34,10 +34,10 @@ import org.apache.hadoop.hbase.HConstants;
import org.apache.hadoop.hbase.TableName; import org.apache.hadoop.hbase.TableName;
import org.apache.hadoop.hbase.classification.InterfaceAudience; import org.apache.hadoop.hbase.classification.InterfaceAudience;
import org.apache.hadoop.hbase.classification.InterfaceStability; 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.Connection;
import org.apache.hadoop.hbase.client.ConnectionFactory; import org.apache.hadoop.hbase.client.ConnectionFactory;
import org.apache.hadoop.hbase.client.Scan; 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.hbase.util.Bytes;
import org.apache.hadoop.mapreduce.Job; import org.apache.hadoop.mapreduce.Job;
import org.apache.hadoop.util.GenericOptionsParser; 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); System.out.println("HFiles will be stored at " + this.bulkloadDir);
HFileOutputFormat2.setOutputPath(job, bulkloadDir); HFileOutputFormat2.setOutputPath(job, bulkloadDir);
try (Connection conn = ConnectionFactory.createConnection(getConf()); try (Connection conn = ConnectionFactory.createConnection(getConf());
Table htable = conn.getTable(TableName.valueOf(dstTableName))) { Admin admin = conn.getAdmin()) {
HFileOutputFormat2.configureIncrementalLoadMap(job, htable); HFileOutputFormat2.configureIncrementalLoadMap(job,
admin.getTableDescriptor((TableName.valueOf(dstTableName))));
} }
} else { } else {
TableMapReduceUtil.initTableMapperJob(tableName, scan, 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(" versions number of cell versions to copy");
System.err.println(" new.name new table's name"); 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(" 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(" families comma-separated list of families to copy");
System.err.println(" To copy from cf1 to cf2, give sourceCfName:destCfName. "); System.err.println(" To copy from cf1 to cf2, give sourceCfName:destCfName. ");
System.err.println(" To keep the same name, just give \"cfName\""); 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) { if (i == args.length-1) {
tableName = cmd; tableName = cmd;
} else { } else {
printUsage("Invalid argument '" + cmd + "'" ); printUsage("Invalid argument '" + cmd + "'");
return false; return false;
} }
} }

View File

@ -449,7 +449,8 @@ public class HFileOutputFormat2
LOG.info("Incremental table " + regionLocator.getName() + " output configured."); 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(); Configuration conf = job.getConfiguration();
job.setOutputKeyClass(ImmutableBytesWritable.class); job.setOutputKeyClass(ImmutableBytesWritable.class);
@ -457,15 +458,14 @@ public class HFileOutputFormat2
job.setOutputFormatClass(HFileOutputFormat2.class); job.setOutputFormatClass(HFileOutputFormat2.class);
// Set compression algorithms based on column families // Set compression algorithms based on column families
configureCompression(conf, table.getTableDescriptor()); configureCompression(conf, tableDescriptor);
configureBloomType(table.getTableDescriptor(), conf); configureBloomType(tableDescriptor, conf);
configureBlockSize(table.getTableDescriptor(), conf); configureBlockSize(tableDescriptor, conf);
HTableDescriptor tableDescriptor = table.getTableDescriptor();
configureDataBlockEncoding(tableDescriptor, conf); configureDataBlockEncoding(tableDescriptor, conf);
TableMapReduceUtil.addDependencyJars(job); TableMapReduceUtil.addDependencyJars(job);
TableMapReduceUtil.initCredentials(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<byte[], Algorithm> compressionMap = new TreeMap<byte[], Map<byte[], Algorithm> compressionMap = new TreeMap<byte[],
Algorithm>(Bytes.BYTES_COMPARATOR); Algorithm>(Bytes.BYTES_COMPARATOR);
for (Map.Entry<byte[], String> e : stringMap.entrySet()) { for (Map.Entry<byte[], String> e : stringMap.entrySet()) {
Algorithm algorithm = AbstractHFileWriter.compressionByName Algorithm algorithm = AbstractHFileWriter.compressionByName(e.getValue());
(e.getValue());
compressionMap.put(e.getKey(), algorithm); compressionMap.put(e.getKey(), algorithm);
} }
return compressionMap; return compressionMap;
@ -602,7 +601,7 @@ public class HFileOutputFormat2
* Serialize column family to compression algorithm map to configuration. * Serialize column family to compression algorithm map to configuration.
* Invoked while configuring the MR job for incremental load. * 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 * @param conf to persist serialized values into
* @throws IOException * @throws IOException
* on failure to read column family descriptors * on failure to read column family descriptors
@ -705,7 +704,7 @@ public class HFileOutputFormat2
* Serialize column family to data block encoding map to configuration. * Serialize column family to data block encoding map to configuration.
* Invoked while configuring the MR job for incremental load. * 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 * @param conf to persist serialized values into
* @throws IOException * @throws IOException
* on failure to read column family descriptors * on failure to read column family descriptors

View File

@ -32,6 +32,7 @@ import org.apache.hadoop.hbase.TableName;
import org.apache.hadoop.hbase.ZooKeeperConnectionException; import org.apache.hadoop.hbase.ZooKeeperConnectionException;
import org.apache.hadoop.hbase.classification.InterfaceAudience; import org.apache.hadoop.hbase.classification.InterfaceAudience;
import org.apache.hadoop.hbase.classification.InterfaceStability; 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.Connection;
import org.apache.hadoop.hbase.client.ConnectionFactory; import org.apache.hadoop.hbase.client.ConnectionFactory;
import org.apache.hadoop.hbase.client.Delete; 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, public static void flushRegionsIfNecessary(Configuration conf) throws IOException,
InterruptedException { InterruptedException {
String tableName = conf.get(TABLE_NAME); String tableName = conf.get(TABLE_NAME);
HBaseAdmin hAdmin = null; Admin hAdmin = null;
Connection connection = null;
String durability = conf.get(WAL_DURABILITY); String durability = conf.get(WAL_DURABILITY);
// Need to flush if the data is written to hbase and skip wal is enabled. // 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 if (conf.get(BULK_OUTPUT_CONF_KEY) == null && durability != null
&& Durability.SKIP_WAL.name().equalsIgnoreCase(durability)) { && Durability.SKIP_WAL.name().equalsIgnoreCase(durability)) {
try { try {
hAdmin = new HBaseAdmin(conf); connection = ConnectionFactory.createConnection(conf);
hAdmin.flush(tableName); hAdmin = connection.getAdmin();
hAdmin.flush(TableName.valueOf(tableName));
} finally { } finally {
if (hAdmin != null) { if (hAdmin != null) {
hAdmin.close(); hAdmin.close();
} }
if (connection != null) {
connection.close();
}
} }
} }
} }

View File

@ -477,7 +477,7 @@ public class ImportTsv extends Configured implements Tool {
job.setInputFormatClass(TextInputFormat.class); job.setInputFormatClass(TextInputFormat.class);
job.setMapperClass(mapperClass); job.setMapperClass(mapperClass);
String hfileOutPath = conf.get(BULK_OUTPUT_CONF_KEY); 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))) { if(StringUtils.isNotEmpty(conf.get(CREDENTIALS_LOCATION))) {
String fileLoc = conf.get(CREDENTIALS_LOCATION); String fileLoc = conf.get(CREDENTIALS_LOCATION);
Credentials cred = Credentials.readTokenStorageFile(new File(fileLoc), conf); 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. // TODO: validation for TsvImporterMapper, not this tool. Move elsewhere.
if (null == getConf().get(MAPPER_CONF_KEY)) { if (null == getConf().get(MAPPER_CONF_KEY)) {
// Make sure columns are specified // Make sure columns are specified
String columns[] = getConf().getStrings(COLUMNS_CONF_KEY); String[] columns = getConf().getStrings(COLUMNS_CONF_KEY);
if (columns == null) { if (columns == null) {
usage("No columns specified. Please specify with -D" + usage("No columns specified. Please specify with -D" +
COLUMNS_CONF_KEY+"=..."); COLUMNS_CONF_KEY+"=...");

View File

@ -47,6 +47,7 @@ import org.apache.hadoop.hbase.classification.InterfaceAudience;
import org.apache.hadoop.hbase.classification.InterfaceStability; import org.apache.hadoop.hbase.classification.InterfaceStability;
import org.apache.hadoop.hbase.client.Admin; import org.apache.hadoop.hbase.client.Admin;
import org.apache.hadoop.hbase.client.Connection; 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.HBaseAdmin;
import org.apache.hadoop.hbase.client.HConnection; import org.apache.hadoop.hbase.client.HConnection;
import org.apache.hadoop.hbase.client.HTable; import org.apache.hadoop.hbase.client.HTable;
@ -932,7 +933,8 @@ public class LoadIncrementalHFiles extends Configured implements Tool {
} }
Path hfofDir = new Path(dirPath); 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); doBulkLoad(hfofDir, table);
return 0; return 0;

View File

@ -29,11 +29,13 @@ import org.apache.hadoop.hbase.classification.InterfaceStability;
import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.hbase.HBaseConfiguration; import org.apache.hadoop.hbase.HBaseConfiguration;
import org.apache.hadoop.hbase.TableName; 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.Delete;
import org.apache.hadoop.hbase.client.HTable;
import org.apache.hadoop.hbase.client.Mutation; import org.apache.hadoop.hbase.client.Mutation;
import org.apache.hadoop.hbase.client.Put; import org.apache.hadoop.hbase.client.Put;
import org.apache.hadoop.hbase.client.Durability; 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.io.ImmutableBytesWritable;
import org.apache.hadoop.hbase.util.Bytes; import org.apache.hadoop.hbase.util.Bytes;
import org.apache.hadoop.mapreduce.JobContext; import org.apache.hadoop.mapreduce.JobContext;
@ -73,7 +75,8 @@ public class MultiTableOutputFormat extends OutputFormat<ImmutableBytesWritable,
protected static class MultiTableRecordWriter extends protected static class MultiTableRecordWriter extends
RecordWriter<ImmutableBytesWritable, Mutation> { RecordWriter<ImmutableBytesWritable, Mutation> {
private static final Log LOG = LogFactory.getLog(MultiTableRecordWriter.class); private static final Log LOG = LogFactory.getLog(MultiTableRecordWriter.class);
Map<ImmutableBytesWritable, HTable> tables; Connection connection;
Map<ImmutableBytesWritable, Table> tables;
Configuration conf; Configuration conf;
boolean useWriteAheadLogging; boolean useWriteAheadLogging;
@ -85,10 +88,10 @@ public class MultiTableOutputFormat extends OutputFormat<ImmutableBytesWritable,
* <tt>false</tt>) to improve performance when bulk loading data. * <tt>false</tt>) to improve performance when bulk loading data.
*/ */
public MultiTableRecordWriter(Configuration conf, public MultiTableRecordWriter(Configuration conf,
boolean useWriteAheadLogging) { boolean useWriteAheadLogging) throws IOException {
LOG.debug("Created new MultiTableRecordReader with WAL " LOG.debug("Created new MultiTableRecordReader with WAL "
+ (useWriteAheadLogging ? "on" : "off")); + (useWriteAheadLogging ? "on" : "off"));
this.tables = new HashMap<ImmutableBytesWritable, HTable>(); this.tables = new HashMap<ImmutableBytesWritable, Table>();
this.conf = conf; this.conf = conf;
this.useWriteAheadLogging = useWriteAheadLogging; this.useWriteAheadLogging = useWriteAheadLogging;
} }
@ -100,10 +103,14 @@ public class MultiTableOutputFormat extends OutputFormat<ImmutableBytesWritable,
* @throws IOException * @throws IOException
* if there is a problem opening a table * if there is a problem opening a table
*/ */
HTable getTable(ImmutableBytesWritable tableName) throws IOException { Table getTable(ImmutableBytesWritable tableName) throws IOException {
if(this.connection == null){
this.connection = ConnectionFactory.createConnection(conf);
}
if (!tables.containsKey(tableName)) { if (!tables.containsKey(tableName)) {
LOG.debug("Opening HTable \"" + Bytes.toString(tableName.get())+ "\" for writing"); LOG.debug("Opening HTable \"" + Bytes.toString(tableName.get())+ "\" for writing");
HTable table = new HTable(conf, TableName.valueOf(tableName.get()));
Table table = connection.getTable(TableName.valueOf(tableName.get()));
table.setAutoFlushTo(false); table.setAutoFlushTo(false);
tables.put(tableName, table); tables.put(tableName, table);
} }
@ -112,9 +119,12 @@ public class MultiTableOutputFormat extends OutputFormat<ImmutableBytesWritable,
@Override @Override
public void close(TaskAttemptContext context) throws IOException { public void close(TaskAttemptContext context) throws IOException {
for (HTable table : tables.values()) { for (Table table : tables.values()) {
table.flushCommits(); table.flushCommits();
} }
if(connection != null){
connection.close();
}
} }
/** /**
@ -129,7 +139,7 @@ public class MultiTableOutputFormat extends OutputFormat<ImmutableBytesWritable,
*/ */
@Override @Override
public void write(ImmutableBytesWritable tableName, Mutation action) throws IOException { public void write(ImmutableBytesWritable tableName, Mutation action) throws IOException {
HTable table = getTable(tableName); Table table = getTable(tableName);
// The actions are not immutable, so we defensively copy them // The actions are not immutable, so we defensively copy them
if (action instanceof Put) { if (action instanceof Put) {
Put put = new Put((Put) action); Put put = new Put((Put) action);

View File

@ -115,7 +115,7 @@ public class WALPlayer extends Configured implements Tool {
@Override @Override
public void setup(Context context) throws IOException { public void setup(Context context) throws IOException {
// only a single table is supported when HFiles are generated with HFileOutputFormat // only a single table is supported when HFiles are generated with HFileOutputFormat
String tables[] = context.getConfiguration().getStrings(TABLES_KEY); String[] tables = context.getConfiguration().getStrings(TABLES_KEY);
if (tables == null || tables.length != 1) { if (tables == null || tables.length != 1) {
// this can only happen when WALMapper is used directly by a class other than WALPlayer // this can only happen when WALMapper is used directly by a class other than WALPlayer
throw new IOException("Exactly one table must be specified for bulk HFile case."); throw new IOException("Exactly one table must be specified for bulk HFile case.");

View File

@ -24,11 +24,15 @@ import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory; import org.apache.commons.logging.LogFactory;
import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.conf.Configured; import org.apache.hadoop.conf.Configured;
import org.apache.hadoop.hbase.*; import org.apache.hadoop.hbase.Abortable;
import org.apache.hadoop.hbase.HBaseConfiguration;
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.HConnectable; import org.apache.hadoop.hbase.client.HConnectable;
import org.apache.hadoop.hbase.client.HConnection; import org.apache.hadoop.hbase.client.HConnection;
import org.apache.hadoop.hbase.client.HConnectionManager; 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.Put;
import org.apache.hadoop.hbase.client.Result; import org.apache.hadoop.hbase.client.Result;
import org.apache.hadoop.hbase.client.ResultScanner; import org.apache.hadoop.hbase.client.ResultScanner;
@ -84,9 +88,13 @@ public class VerifyReplication extends Configured implements Tool {
public static class Verifier public static class Verifier
extends TableMapper<ImmutableBytesWritable, Put> { extends TableMapper<ImmutableBytesWritable, Put> {
public static enum Counters { public static enum Counters {
GOODROWS, BADROWS, ONLY_IN_SOURCE_TABLE_ROWS, ONLY_IN_PEER_TABLE_ROWS, CONTENT_DIFFERENT_ROWS} 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 ResultScanner replicatedScanner;
private Result currentCompareRowInPeerTable; private Result currentCompareRowInPeerTable;
@ -129,8 +137,8 @@ public class VerifyReplication extends Configured implements Tool {
ZKUtil.applyClusterKeyToConf(peerConf, zkClusterKey); ZKUtil.applyClusterKeyToConf(peerConf, zkClusterKey);
TableName tableName = TableName.valueOf(conf.get(NAME + ".tableName")); TableName tableName = TableName.valueOf(conf.get(NAME + ".tableName"));
// TODO: THis HTable doesn't get closed. Fix! connection = ConnectionFactory.createConnection(peerConf);
Table replicatedTable = new HTable(peerConf, tableName); replicatedTable = connection.getTable(tableName);
scan.setStartRow(value.getRow()); scan.setStartRow(value.getRow());
scan.setStopRow(tableSplit.getEndRow()); scan.setStopRow(tableSplit.getEndRow());
replicatedScanner = replicatedTable.getScanner(scan); replicatedScanner = replicatedTable.getScanner(scan);
@ -191,6 +199,20 @@ public class VerifyReplication extends Configured implements Tool {
replicatedScanner = null; 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);
}
}
} }
} }

View File

@ -37,12 +37,13 @@ import org.apache.hadoop.hbase.MetaTableAccessor;
import org.apache.hadoop.hbase.TableName; import org.apache.hadoop.hbase.TableName;
import org.apache.hadoop.hbase.TableNotDisabledException; import org.apache.hadoop.hbase.TableNotDisabledException;
import org.apache.hadoop.hbase.client.Admin; 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.Delete;
import org.apache.hadoop.hbase.client.HBaseAdmin; import org.apache.hadoop.hbase.client.HBaseAdmin;
import org.apache.hadoop.hbase.client.HConnectable; import org.apache.hadoop.hbase.client.HConnectable;
import org.apache.hadoop.hbase.client.HConnection; import org.apache.hadoop.hbase.client.HConnection;
import org.apache.hadoop.hbase.client.HConnectionManager; 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.Result;
import org.apache.hadoop.hbase.client.ResultScanner; import org.apache.hadoop.hbase.client.ResultScanner;
import org.apache.hadoop.hbase.client.Table; import org.apache.hadoop.hbase.client.Table;
@ -245,7 +246,8 @@ class HMerge {
throws IOException { throws IOException {
super(conf, fs, tableName); super(conf, fs, tableName);
this.tableName = 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, this.metaScanner = table.getScanner(HConstants.CATALOG_FAMILY,
HConstants.REGIONINFO_QUALIFIER); HConstants.REGIONINFO_QUALIFIER);
this.latestRegion = null; this.latestRegion = null;

View File

@ -24,7 +24,6 @@
import="org.apache.hadoop.conf.Configuration" import="org.apache.hadoop.conf.Configuration"
import="org.apache.hadoop.hbase.client.HTable" import="org.apache.hadoop.hbase.client.HTable"
import="org.apache.hadoop.hbase.client.Admin" 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.HRegionInfo"
import="org.apache.hadoop.hbase.ServerName" import="org.apache.hadoop.hbase.ServerName"
import="org.apache.hadoop.hbase.ServerLoad" import="org.apache.hadoop.hbase.ServerLoad"
@ -33,17 +32,16 @@
import="org.apache.hadoop.hbase.zookeeper.MetaTableLocator" import="org.apache.hadoop.hbase.zookeeper.MetaTableLocator"
import="org.apache.hadoop.hbase.util.Bytes" import="org.apache.hadoop.hbase.util.Bytes"
import="org.apache.hadoop.hbase.util.FSUtils" 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.protobuf.generated.AdminProtos.GetRegionInfoResponse.CompactionState"
import="org.apache.hadoop.hbase.TableName" import="org.apache.hadoop.hbase.TableName"
import="org.apache.hadoop.hbase.client.RegionReplicaUtil"
import="org.apache.hadoop.hbase.HBaseConfiguration" %> import="org.apache.hadoop.hbase.HBaseConfiguration" %>
<% <%
HMaster master = (HMaster)getServletContext().getAttribute(HMaster.MASTER); HMaster master = (HMaster)getServletContext().getAttribute(HMaster.MASTER);
Configuration conf = master.getConfiguration(); Configuration conf = master.getConfiguration();
MetaTableLocator metaTableLocator = new MetaTableLocator(); MetaTableLocator metaTableLocator = new MetaTableLocator();
String fqtn = request.getParameter("name"); String fqtn = request.getParameter("name");
HTable table = new HTable(conf, fqtn); HTable table = (HTable) master.getConnection().getTable(fqtn);
String tableHeader; String tableHeader;
boolean withReplica = false; boolean withReplica = false;
if (table.getTableDescriptor().getRegionReplication() > 1) { if (table.getTableDescriptor().getRegionReplication() > 1) {

View File

@ -17,34 +17,6 @@
*/ */
package org.apache.hadoop.hbase; 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.Log;
import org.apache.commons.logging.LogFactory; import org.apache.commons.logging.LogFactory;
import org.apache.commons.logging.impl.Jdk14Logger; 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.InternalScanner;
import org.apache.hadoop.hbase.regionserver.RegionServerServices; import org.apache.hadoop.hbase.regionserver.RegionServerServices;
import org.apache.hadoop.hbase.regionserver.RegionServerStoppedException; 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.security.User;
import org.apache.hadoop.hbase.tool.Canary; import org.apache.hadoop.hbase.tool.Canary;
import org.apache.hadoop.hbase.util.Bytes; 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.RegionSplitter;
import org.apache.hadoop.hbase.util.RetryCounter; import org.apache.hadoop.hbase.util.RetryCounter;
import org.apache.hadoop.hbase.util.Threads; 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.EmptyWatcher;
import org.apache.hadoop.hbase.zookeeper.MiniZooKeeperCluster; import org.apache.hadoop.hbase.zookeeper.MiniZooKeeperCluster;
import org.apache.hadoop.hbase.zookeeper.ZKConfig; 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;
import org.apache.zookeeper.ZooKeeper.States; 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 * Facility for testing HBase. Replacement for
* old HBaseTestCase and HBaseClusterTestCase functionality. * old HBaseTestCase and HBaseClusterTestCase functionality.
@ -976,7 +976,7 @@ public class HBaseTestingUtility extends HBaseCommonTestingUtility {
this.hbaseCluster = this.hbaseCluster =
new MiniHBaseCluster(c, numMasters, numSlaves, masterClass, regionserverClass); new MiniHBaseCluster(c, numMasters, numSlaves, masterClass, regionserverClass);
// Don't leave here till we've done a successful scan of the hbase:meta // 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()); ResultScanner s = t.getScanner(new Scan());
while (s.next() != null) { while (s.next() != null) {
continue; continue;
@ -996,6 +996,10 @@ public class HBaseTestingUtility extends HBaseCommonTestingUtility {
* @throws IOException * @throws IOException
*/ */
public void restartHBaseCluster(int servers) throws IOException, InterruptedException { public void restartHBaseCluster(int servers) throws IOException, InterruptedException {
if(connection != null){
connection.close();
connection = null;
}
this.hbaseCluster = new MiniHBaseCluster(this.conf, servers); this.hbaseCluster = new MiniHBaseCluster(this.conf, servers);
// Don't leave here till we've done a successful scan of the hbase:meta // 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); 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); getHBaseAdmin().createTable(desc, startKey, endKey, numRegions);
// HBaseAdmin only waits for regions to appear in hbase:meta we should wait until they are assigned // HBaseAdmin only waits for regions to appear in hbase:meta we should wait until they are assigned
waitUntilAllRegionsAssigned(tableName); 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); getHBaseAdmin().createTable(htd, splitRows);
// HBaseAdmin only waits for regions to appear in hbase:meta we should wait until they are assigned // HBaseAdmin only waits for regions to appear in hbase:meta we should wait until they are assigned
waitUntilAllRegionsAssigned(htd.getTableName()); 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); desc.addFamily(hcd);
} }
getHBaseAdmin().createTable(desc); 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); getHBaseAdmin().createTable(desc);
// HBaseAdmin only waits for regions to appear in hbase:meta we should wait until they are assigned // HBaseAdmin only waits for regions to appear in hbase:meta we should wait until they are assigned
waitUntilAllRegionsAssigned(tableName); waitUntilAllRegionsAssigned(tableName);
return new HTable(c, tableName); return (HTable) getConnection().getTable(tableName);
} }
/** /**
@ -1399,7 +1403,7 @@ public class HBaseTestingUtility extends HBaseCommonTestingUtility {
desc.addFamily(hcd); desc.addFamily(hcd);
} }
getHBaseAdmin().createTable(desc); 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); getHBaseAdmin().createTable(desc);
// HBaseAdmin only waits for regions to appear in hbase:meta we should wait until they are assigned // HBaseAdmin only waits for regions to appear in hbase:meta we should wait until they are assigned
waitUntilAllRegionsAssigned(tableName); 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); getHBaseAdmin().createTable(desc);
// HBaseAdmin only waits for regions to appear in hbase:meta we should wait until they are assigned // HBaseAdmin only waits for regions to appear in hbase:meta we should wait until they are assigned
waitUntilAllRegionsAssigned(tableName); 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); getHBaseAdmin().createTable(desc);
// HBaseAdmin only waits for regions to appear in hbase:meta we should wait until they are assigned // HBaseAdmin only waits for regions to appear in hbase:meta we should wait until they are assigned
waitUntilAllRegionsAssigned(tableName); 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); getHBaseAdmin().createTable(desc, splitRows);
// HBaseAdmin only waits for regions to appear in hbase:meta we should wait until they are assigned // HBaseAdmin only waits for regions to appear in hbase:meta we should wait until they are assigned
waitUntilAllRegionsAssigned(tableName); 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); getHBaseAdmin().createTable(desc, splitRows);
// HBaseAdmin only waits for regions to appear in hbase:meta we should wait until they are assigned // HBaseAdmin only waits for regions to appear in hbase:meta we should wait until they are assigned
waitUntilAllRegionsAssigned(desc.getTableName()); 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 * @throws IOException
*/ */
public HTable deleteTableData(TableName tableName) 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(); Scan scan = new Scan();
ResultScanner resScan = table.getScanner(scan); ResultScanner resScan = table.getScanner(scan);
for(Result res : resScan) { for(Result res : resScan) {
@ -1831,7 +1835,7 @@ public class HBaseTestingUtility extends HBaseCommonTestingUtility {
public HTable truncateTable(final TableName tableName, final boolean preserveRegions) throws IOException { public HTable truncateTable(final TableName tableName, final boolean preserveRegions) throws IOException {
Admin admin = getHBaseAdmin(); Admin admin = getHBaseAdmin();
admin.truncateTable(tableName, preserveRegions); 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<HRegionInfo> createMultiRegionsInMeta(final Configuration conf, public List<HRegionInfo> createMultiRegionsInMeta(final Configuration conf,
final HTableDescriptor htd, byte [][] startKeys) final HTableDescriptor htd, byte [][] startKeys)
throws IOException { 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); Arrays.sort(startKeys, Bytes.BYTES_COMPARATOR);
List<HRegionInfo> newRegions = new ArrayList<HRegionInfo>(startKeys.length); List<HRegionInfo> newRegions = new ArrayList<HRegionInfo>(startKeys.length);
// add custom ones // add custom ones
@ -2294,7 +2298,7 @@ public class HBaseTestingUtility extends HBaseCommonTestingUtility {
*/ */
public List<byte[]> getMetaTableRows() throws IOException { public List<byte[]> getMetaTableRows() throws IOException {
// TODO: Redo using MetaTableAccessor class // 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<byte[]> rows = new ArrayList<byte[]>(); List<byte[]> rows = new ArrayList<byte[]>();
ResultScanner s = t.getScanner(new Scan()); ResultScanner s = t.getScanner(new Scan());
for (Result result : s) { for (Result result : s) {
@ -2314,7 +2318,7 @@ public class HBaseTestingUtility extends HBaseCommonTestingUtility {
*/ */
public List<byte[]> getMetaTableRows(TableName tableName) throws IOException { public List<byte[]> getMetaTableRows(TableName tableName) throws IOException {
// TODO: Redo using MetaTableAccessor. // 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<byte[]> rows = new ArrayList<byte[]>(); List<byte[]> rows = new ArrayList<byte[]>();
ResultScanner s = t.getScanner(new Scan()); ResultScanner s = t.getScanner(new Scan());
for (Result result : s) { for (Result result : s) {
@ -2631,7 +2635,7 @@ public class HBaseTestingUtility extends HBaseCommonTestingUtility {
monitor.close(); monitor.close();
if (checkStatus) { 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) public void waitUntilAllRegionsAssigned(final TableName tableName, final long timeout)
throws IOException { throws IOException {
final Table meta = new HTable(getConfiguration(), TableName.META_TABLE_NAME); final Table meta = getConnection().getTable(TableName.META_TABLE_NAME);
try { try {
waitFor(timeout, 200, true, new Predicate<IOException>() { waitFor(timeout, 200, true, new Predicate<IOException>() {
@Override @Override
@ -3548,9 +3552,8 @@ public class HBaseTestingUtility extends HBaseCommonTestingUtility {
return totalNumberOfRegions; return totalNumberOfRegions;
} }
public static int getMetaRSPort(Configuration conf) throws IOException { public static int getMetaRSPort(Connection connection) throws IOException {
try (Connection c = ConnectionFactory.createConnection(); try (RegionLocator locator = connection.getRegionLocator(TableName.META_TABLE_NAME)) {
RegionLocator locator = c.getRegionLocator(TableName.META_TABLE_NAME)) {
return locator.getRegionLocation(Bytes.toBytes("")).getPort(); return locator.getRegionLocation(Bytes.toBytes("")).getPort();
} }
} }

View File

@ -1020,7 +1020,7 @@ public class PerformanceEvaluation extends Configured implements Tool {
if (!opts.oneCon) { if (!opts.oneCon) {
this.connection = ConnectionFactory.createConnection(conf); 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); this.table.setAutoFlushTo(opts.autoFlush);
latency = YammerHistogramUtils.newHistogram(new UniformSample(1024 * 500)); latency = YammerHistogramUtils.newHistogram(new UniformSample(1024 * 500));
valueSize = 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. // the TestOptions introspection for us and dump the output in a readable format.
LOG.info(cmd.getSimpleName() + " test run options=" + MAPPER.writeValueAsString(opts)); LOG.info(cmd.getSimpleName() + " test run options=" + MAPPER.writeValueAsString(opts));
Admin admin = null; Admin admin = null;
Connection connection = null;
try { try {
admin = new HBaseAdmin(getConf()); connection = ConnectionFactory.createConnection(getConf());
admin = connection.getAdmin();
checkTable(admin, opts); checkTable(admin, opts);
} finally { } finally {
if (admin != null) admin.close(); if (admin != null) admin.close();
if (connection != null) connection.close();
} }
if (opts.nomapred) { if (opts.nomapred) {
doLocalClients(opts, getConf()); doLocalClients(opts, getConf());

View File

@ -27,6 +27,8 @@ import org.apache.hadoop.fs.FSDataInputStream;
import org.apache.hadoop.fs.FileSystem; import org.apache.hadoop.fs.FileSystem;
import org.apache.hadoop.fs.Path; import org.apache.hadoop.fs.Path;
import org.apache.hadoop.hbase.classification.InterfaceAudience; 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.HTable;
import org.apache.hadoop.hbase.client.Result; import org.apache.hadoop.hbase.client.Result;
import org.apache.hadoop.hbase.client.ResultScanner; import org.apache.hadoop.hbase.client.ResultScanner;
@ -149,7 +151,8 @@ public class ScanPerformanceEvaluation extends AbstractHBaseTool {
Stopwatch scanTimer = new Stopwatch(); Stopwatch scanTimer = new Stopwatch();
tableOpenTimer.start(); tableOpenTimer.start();
Table table = new HTable(getConf(), TableName.valueOf(tablename)); Connection connection = ConnectionFactory.createConnection(getConf());
Table table = connection.getTable(TableName.valueOf(tablename));
tableOpenTimer.stop(); tableOpenTimer.stop();
Scan scan = getScan(); Scan scan = getScan();
@ -172,6 +175,7 @@ public class ScanPerformanceEvaluation extends AbstractHBaseTool {
scanTimer.stop(); scanTimer.stop();
scanner.close(); scanner.close();
table.close(); table.close();
connection.close();
ScanMetrics metrics = ProtobufUtil.toScanMetrics(scan.getAttribute(Scan.SCAN_ATTRIBUTES_METRICS_DATA)); ScanMetrics metrics = ProtobufUtil.toScanMetrics(scan.getAttribute(Scan.SCAN_ATTRIBUTES_METRICS_DATA));
long totalBytes = metrics.countOfBytesInResults.get(); long totalBytes = metrics.countOfBytesInResults.get();

View File

@ -28,6 +28,8 @@ import org.apache.commons.logging.LogFactory;
import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.hbase.MultithreadedTestUtil.RepeatingTestThread; import org.apache.hadoop.hbase.MultithreadedTestUtil.RepeatingTestThread;
import org.apache.hadoop.hbase.MultithreadedTestUtil.TestContext; 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.Get;
import org.apache.hadoop.hbase.client.HBaseAdmin; import org.apache.hadoop.hbase.client.HBaseAdmin;
import org.apache.hadoop.hbase.client.HTable; import org.apache.hadoop.hbase.client.HTable;
@ -113,7 +115,8 @@ public class TestAcidGuarantees implements Tool {
super(ctx); super(ctx);
this.targetRows = targetRows; this.targetRows = targetRows;
this.targetFamilies = targetFamilies; 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 { public void doAnAction() throws Exception {
// Pick a random row to write into // Pick a random row to write into
@ -148,7 +151,8 @@ public class TestAcidGuarantees implements Tool {
super(ctx); super(ctx);
this.targetRow = targetRow; this.targetRow = targetRow;
this.targetFamilies = targetFamilies; 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 { public void doAnAction() throws Exception {
@ -205,7 +209,8 @@ public class TestAcidGuarantees implements Tool {
byte targetFamilies[][]) throws IOException { byte targetFamilies[][]) throws IOException {
super(ctx); super(ctx);
this.targetFamilies = targetFamilies; 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 { public void doAnAction() throws Exception {

View File

@ -67,7 +67,7 @@ public class TestInfoServers {
@Test @Test
public void testInfoServersRedirect() throws Exception { public void testInfoServersRedirect() throws Exception {
// give the cluster time to start up // 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(); int port = UTIL.getHBaseCluster().getMaster().getInfoServer().getPort();
assertContainsContent(new URL("http://localhost:" + port + assertContainsContent(new URL("http://localhost:" + port +
"/index.html"), "master-status"); "/index.html"), "master-status");
@ -87,7 +87,7 @@ public class TestInfoServers {
@Test @Test
public void testInfoServersStatusPages() throws Exception { public void testInfoServersStatusPages() throws Exception {
// give the cluster time to start up // 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(); int port = UTIL.getHBaseCluster().getMaster().getInfoServer().getPort();
assertContainsContent(new URL("http://localhost:" + port + assertContainsContent(new URL("http://localhost:" + port +
"/master-status"), "meta"); "/master-status"), "meta");
@ -102,7 +102,7 @@ public class TestInfoServers {
TableName tableName = TableName.valueOf("testMasterServerReadOnly"); TableName tableName = TableName.valueOf("testMasterServerReadOnly");
byte[] cf = Bytes.toBytes("d"); byte[] cf = Bytes.toBytes("d");
UTIL.createTable(tableName, cf); UTIL.createTable(tableName, cf);
new HTable(UTIL.getConfiguration(), tableName).close(); UTIL.getConnection().getTable(tableName).close();
int port = UTIL.getHBaseCluster().getMaster().getInfoServer().getPort(); int port = UTIL.getHBaseCluster().getMaster().getInfoServer().getPort();
assertDoesNotContainContent( assertDoesNotContainContent(
new URL("http://localhost:" + port + "/table.jsp?name=" + tableName + "&action=split&key="), new URL("http://localhost:" + port + "/table.jsp?name=" + tableName + "&action=split&key="),

View File

@ -45,6 +45,7 @@ import org.apache.hadoop.hbase.client.Table;
import org.apache.hadoop.hbase.testclassification.MediumTests; import org.apache.hadoop.hbase.testclassification.MediumTests;
import org.apache.hadoop.hbase.testclassification.MiscTests; import org.apache.hadoop.hbase.testclassification.MiscTests;
import org.apache.hadoop.hbase.util.Bytes; import org.apache.hadoop.hbase.util.Bytes;
import org.apache.hadoop.hbase.util.Pair;
import org.junit.After; import org.junit.After;
import org.junit.AfterClass; import org.junit.AfterClass;
import org.junit.Before; import org.junit.Before;
@ -77,12 +78,7 @@ public class TestMultiVersions {
@Before @Before
public void before() public void before()
throws MasterNotRunningException, ZooKeeperConnectionException, IOException { throws MasterNotRunningException, ZooKeeperConnectionException, IOException {
this.admin = new HBaseAdmin(UTIL.getConfiguration()); this.admin = UTIL.getHBaseAdmin();
}
@After
public void after() throws IOException {
this.admin.close();
} }
/** /**
@ -101,7 +97,7 @@ public class TestMultiVersions {
hcd.setMaxVersions(3); hcd.setMaxVersions(3);
desc.addFamily(hcd); desc.addFamily(hcd);
this.admin.createTable(desc); 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 // TODO: Remove these deprecated classes or pull them in here if this is
// only test using them. // only test using them.
Incommon incommon = new HTableIncommon(table); Incommon incommon = new HTableIncommon(table);
@ -144,16 +140,15 @@ public class TestMultiVersions {
this.admin.createTable(desc); this.admin.createTable(desc);
Put put = new Put(row, timestamp1); Put put = new Put(row, timestamp1);
put.add(contents, contents, value1); put.add(contents, contents, value1);
Table table = new HTable(UTIL.getConfiguration(), desc.getTableName()); Table table = UTIL.getConnection().getTable(desc.getTableName());
table.put(put); table.put(put);
// Shut down and restart the HBase cluster // Shut down and restart the HBase cluster
table.close(); table.close();
UTIL.shutdownMiniHBaseCluster(); UTIL.shutdownMiniHBaseCluster();
LOG.debug("HBase cluster shut down -- restarting"); LOG.debug("HBase cluster shut down -- restarting");
UTIL.startMiniHBaseCluster(1, NUM_SLAVES); UTIL.startMiniHBaseCluster(1, NUM_SLAVES);
// Make a new connection. Use new Configuration instance because old one // Make a new connection.
// is tied to an HConnection that has since gone stale. table = UTIL.getConnection().getTable(desc.getTableName());
table = new HTable(new Configuration(UTIL.getConfiguration()), desc.getTableName());
// Overwrite previous value // Overwrite previous value
put = new Put(row, timestamp2); put = new Put(row, timestamp2);
put.add(contents, contents, value2); put.add(contents, contents, value2);
@ -207,23 +202,25 @@ public class TestMultiVersions {
final byte [][] splitRows = new byte[][] {Bytes.toBytes("row_0500")}; final byte [][] splitRows = new byte[][] {Bytes.toBytes("row_0500")};
final long [] timestamp = new long[] {100L, 1000L}; final long [] timestamp = new long[] {100L, 1000L};
this.admin.createTable(desc, splitRows); 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. // Assert we got the region layout wanted.
NavigableMap<HRegionInfo, ServerName> locations = table.getRegionLocations(); Pair<byte[][], byte[][]> keys = UTIL.getConnection()
assertEquals(2, locations.size()); .getRegionLocator(tableName).getStartEndKeys();
int index = 0; assertEquals(2, keys.getFirst().length);
for (HRegionInfo hri: locations.keySet()) { byte[][] startKeys = keys.getFirst();
if (index == 0) { byte[][] endKeys = keys.getSecond();
assertTrue(Bytes.equals(HConstants.EMPTY_START_ROW, hri.getStartKey()));
assertTrue(Bytes.equals(hri.getEndKey(), splitRows[0])); for (int i = 0; i < startKeys.length; i++) {
} else if (index == 1) { if (i == 0) {
assertTrue(Bytes.equals(splitRows[0], hri.getStartKey())); assertTrue(Bytes.equals(HConstants.EMPTY_START_ROW, startKeys[i]));
assertTrue(Bytes.equals(hri.getEndKey(), HConstants.EMPTY_END_ROW)); 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 // 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++) { for (int j = 0; j < timestamp.length; j++) {
Put put = new Put(rows[i], timestamp[j]); Put put = new Put(rows[i], timestamp[j]);
put.add(HConstants.CATALOG_FAMILY, null, timestamp[j], put.add(HConstants.CATALOG_FAMILY, null, timestamp[j],

View File

@ -252,7 +252,7 @@ public class TestNamespace {
} }
//sanity check try to write and read from table //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")); Put p = new Put(Bytes.toBytes("row1"));
p.add(Bytes.toBytes("my_cf"),Bytes.toBytes("my_col"),Bytes.toBytes("value1")); p.add(Bytes.toBytes("my_cf"),Bytes.toBytes("my_col"),Bytes.toBytes("value1"));
table.put(p); table.put(p);

View File

@ -35,11 +35,11 @@ import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory; import org.apache.commons.logging.LogFactory;
import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.hbase.client.Admin; 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.Get;
import org.apache.hadoop.hbase.client.HBaseAdmin; 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.HConnectionManager;
import org.apache.hadoop.hbase.client.HTable;
import org.apache.hadoop.hbase.client.Put; import org.apache.hadoop.hbase.client.Put;
import org.apache.hadoop.hbase.client.Result; import org.apache.hadoop.hbase.client.Result;
import org.apache.hadoop.hbase.client.ResultScanner; 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 { throws NoSuchMethodException, InvocationTargetException, IllegalAccessException {
Method getterZK = c.getClass().getDeclaredMethod("getKeepAliveZooKeeperWatcher"); Method getterZK = c.getClass().getDeclaredMethod("getKeepAliveZooKeeperWatcher");
getterZK.setAccessible(true); getterZK.setAccessible(true);
@ -148,7 +148,7 @@ public class TestZooKeeper {
// We don't want to share the connection as we will check its state // We don't want to share the connection as we will check its state
c.set(HConstants.HBASE_CLIENT_INSTANCE_ID, "1111"); c.set(HConstants.HBASE_CLIENT_INSTANCE_ID, "1111");
HConnection connection = HConnectionManager.getConnection(c); Connection connection = ConnectionFactory.createConnection(c);
ZooKeeperWatcher connectionZK = getZooKeeperWatcher(connection); ZooKeeperWatcher connectionZK = getZooKeeperWatcher(connection);
LOG.info("ZooKeeperWatcher= 0x"+ Integer.toHexString( LOG.info("ZooKeeperWatcher= 0x"+ Integer.toHexString(
@ -253,15 +253,14 @@ public class TestZooKeeper {
HColumnDescriptor family = new HColumnDescriptor("fam"); HColumnDescriptor family = new HColumnDescriptor("fam");
desc.addFamily(family); desc.addFamily(family);
LOG.info("Creating table " + tableName); LOG.info("Creating table " + tableName);
Admin admin = new HBaseAdmin(TEST_UTIL.getConfiguration()); Admin admin = TEST_UTIL.getHBaseAdmin();
try { try {
admin.createTable(desc); admin.createTable(desc);
} finally { } finally {
admin.close(); admin.close();
} }
Table table = Table table = TEST_UTIL.getConnection().getTable(desc.getTableName());
new HTable(new Configuration(TEST_UTIL.getConfiguration()), desc.getTableName());
Put put = new Put(Bytes.toBytes("testrow")); Put put = new Put(Bytes.toBytes("testrow"));
put.add(Bytes.toBytes("fam"), put.add(Bytes.toBytes("fam"),
Bytes.toBytes("col"), Bytes.toBytes("testdata")); Bytes.toBytes("col"), Bytes.toBytes("testdata"));
@ -273,11 +272,11 @@ public class TestZooKeeper {
@Test @Test
public void testMultipleZK() public void testMultipleZK()
throws IOException, NoSuchMethodException, InvocationTargetException, IllegalAccessException { throws IOException, NoSuchMethodException, InvocationTargetException, IllegalAccessException {
Table localMeta = Table localMeta = TEST_UTIL.getConnection().getTable(TableName.META_TABLE_NAME);
new HTable(new Configuration(TEST_UTIL.getConfiguration()), TableName.META_TABLE_NAME);
Configuration otherConf = new Configuration(TEST_UTIL.getConfiguration()); Configuration otherConf = new Configuration(TEST_UTIL.getConfiguration());
otherConf.set(HConstants.ZOOKEEPER_QUORUM, "127.0.0.1"); 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 // dummy, just to open the connection
final byte [] row = new byte [] {'r'}; final byte [] row = new byte [] {'r'};
@ -294,6 +293,7 @@ public class TestZooKeeper {
localMeta.close(); localMeta.close();
ipMeta.close(); ipMeta.close();
connection.close();
} }
/** /**
@ -494,7 +494,7 @@ public class TestZooKeeper {
ZooKeeperWatcher zkw = m.getZooKeeper(); ZooKeeperWatcher zkw = m.getZooKeeper();
int expectedNumOfListeners = zkw.getNumberOfListeners(); int expectedNumOfListeners = zkw.getNumberOfListeners();
// now the cluster is up. So assign some regions. // now the cluster is up. So assign some regions.
Admin admin = new HBaseAdmin(TEST_UTIL.getConfiguration()); Admin admin = TEST_UTIL.getHBaseAdmin();
try { try {
byte[][] SPLIT_KEYS = new byte[][] { Bytes.toBytes("a"), Bytes.toBytes("b"), byte[][] SPLIT_KEYS = new byte[][] { Bytes.toBytes("a"), Bytes.toBytes("b"),
Bytes.toBytes("c"), Bytes.toBytes("d"), Bytes.toBytes("e"), Bytes.toBytes("f"), Bytes.toBytes("c"), Bytes.toBytes("d"), Bytes.toBytes("e"), Bytes.toBytes("f"),
@ -531,7 +531,7 @@ public class TestZooKeeper {
cluster.startRegionServer(); cluster.startRegionServer();
HMaster m = cluster.getMaster(); HMaster m = cluster.getMaster();
// now the cluster is up. So assign some regions. // now the cluster is up. So assign some regions.
Admin admin = new HBaseAdmin(TEST_UTIL.getConfiguration()); Admin admin = TEST_UTIL.getHBaseAdmin();
Table table = null; Table table = null;
try { try {
byte[][] SPLIT_KEYS = new byte[][] { Bytes.toBytes("1"), Bytes.toBytes("2"), byte[][] SPLIT_KEYS = new byte[][] { Bytes.toBytes("1"), Bytes.toBytes("2"),
@ -543,7 +543,7 @@ public class TestZooKeeper {
htd.addFamily(hcd); htd.addFamily(hcd);
admin.createTable(htd, SPLIT_KEYS); admin.createTable(htd, SPLIT_KEYS);
TEST_UTIL.waitUntilNoRegionsInTransition(60000); TEST_UTIL.waitUntilNoRegionsInTransition(60000);
table = new HTable(TEST_UTIL.getConfiguration(), htd.getTableName()); table = TEST_UTIL.getConnection().getTable(htd.getTableName());
Put p; Put p;
int numberOfPuts; int numberOfPuts;
for (numberOfPuts = 0; numberOfPuts < 6; numberOfPuts++) { for (numberOfPuts = 0; numberOfPuts < 6; numberOfPuts++) {

View File

@ -367,7 +367,7 @@ public class TestAdmin1 {
splitKeys[1] = Bytes.toBytes(8); splitKeys[1] = Bytes.toBytes(8);
// Create & Fill the table // 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 { try {
TEST_UTIL.loadNumericRows(table, HConstants.CATALOG_FAMILY, 0, 10); TEST_UTIL.loadNumericRows(table, HConstants.CATALOG_FAMILY, 0, 10);
assertEquals(10, TEST_UTIL.countRows(table)); assertEquals(10, TEST_UTIL.countRows(table));
@ -379,7 +379,7 @@ public class TestAdmin1 {
// Truncate & Verify // Truncate & Verify
this.admin.disableTable(tableName); this.admin.disableTable(tableName);
this.admin.truncateTable(tableName, preserveSplits); this.admin.truncateTable(tableName, preserveSplits);
table = new HTable(TEST_UTIL.getConfiguration(), tableName); table = TEST_UTIL.getConnection().getTable(tableName);
try { try {
assertEquals(0, TEST_UTIL.countRows(table)); assertEquals(0, TEST_UTIL.countRows(table));
} finally { } finally {
@ -402,7 +402,7 @@ public class TestAdmin1 {
htd.addFamily(fam2); htd.addFamily(fam2);
htd.addFamily(fam3); htd.addFamily(fam3);
this.admin.createTable(htd); 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(); HTableDescriptor confirmedHtd = table.getTableDescriptor();
assertEquals(htd.compareTo(confirmedHtd), 0); assertEquals(htd.compareTo(confirmedHtd), 0);
table.close(); table.close();
@ -585,7 +585,7 @@ public class TestAdmin1 {
HTableDescriptor desc = new HTableDescriptor(tableName); HTableDescriptor desc = new HTableDescriptor(tableName);
desc.addFamily(new HColumnDescriptor(HConstants.CATALOG_FAMILY)); desc.addFamily(new HColumnDescriptor(HConstants.CATALOG_FAMILY));
admin.createTable(desc); admin.createTable(desc);
HTable ht = new HTable(TEST_UTIL.getConfiguration(), tableName); HTable ht = (HTable) TEST_UTIL.getConnection().getTable(tableName);
Map<HRegionInfo, ServerName> regions = ht.getRegionLocations(); Map<HRegionInfo, ServerName> regions = ht.getRegionLocations();
assertEquals("Table should have only 1 region", 1, regions.size()); assertEquals("Table should have only 1 region", 1, regions.size());
ht.close(); ht.close();
@ -594,7 +594,7 @@ public class TestAdmin1 {
desc = new HTableDescriptor(TABLE_2); desc = new HTableDescriptor(TABLE_2);
desc.addFamily(new HColumnDescriptor(HConstants.CATALOG_FAMILY)); desc.addFamily(new HColumnDescriptor(HConstants.CATALOG_FAMILY));
admin.createTable(desc, new byte[][]{new byte[]{42}}); 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(); regions = ht2.getRegionLocations();
assertEquals("Table should have only 2 region", 2, regions.size()); assertEquals("Table should have only 2 region", 2, regions.size());
ht2.close(); ht2.close();
@ -603,7 +603,7 @@ public class TestAdmin1 {
desc = new HTableDescriptor(TABLE_3); desc = new HTableDescriptor(TABLE_3);
desc.addFamily(new HColumnDescriptor(HConstants.CATALOG_FAMILY)); desc.addFamily(new HColumnDescriptor(HConstants.CATALOG_FAMILY));
admin.createTable(desc, "a".getBytes(), "z".getBytes(), 3); 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(); regions = ht3.getRegionLocations();
assertEquals("Table should have only 3 region", 3, regions.size()); assertEquals("Table should have only 3 region", 3, regions.size());
ht3.close(); ht3.close();
@ -622,7 +622,7 @@ public class TestAdmin1 {
desc = new HTableDescriptor(TABLE_5); desc = new HTableDescriptor(TABLE_5);
desc.addFamily(new HColumnDescriptor(HConstants.CATALOG_FAMILY)); desc.addFamily(new HColumnDescriptor(HConstants.CATALOG_FAMILY));
admin.createTable(desc, new byte[] {1}, new byte[] {127}, 16); 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(); regions = ht5.getRegionLocations();
assertEquals("Table should have 16 region", 16, regions.size()); assertEquals("Table should have 16 region", 16, regions.size());
ht5.close(); ht5.close();
@ -653,7 +653,7 @@ public class TestAdmin1 {
boolean tableAvailable = admin.isTableAvailable(tableName, splitKeys); boolean tableAvailable = admin.isTableAvailable(tableName, splitKeys);
assertTrue("Table should be created with splitKyes + 1 rows in META", tableAvailable); 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<HRegionInfo, ServerName> regions = ht.getRegionLocations(); Map<HRegionInfo, ServerName> regions = ht.getRegionLocations();
assertEquals("Tried to create " + expectedRegions + " regions " + assertEquals("Tried to create " + expectedRegions + " regions " +
"but only found " + regions.size(), "but only found " + regions.size(),
@ -710,10 +710,10 @@ public class TestAdmin1 {
desc = new HTableDescriptor(TABLE_2); desc = new HTableDescriptor(TABLE_2);
desc.addFamily(new HColumnDescriptor(HConstants.CATALOG_FAMILY)); desc.addFamily(new HColumnDescriptor(HConstants.CATALOG_FAMILY));
admin = new HBaseAdmin(TEST_UTIL.getConfiguration()); admin = TEST_UTIL.getHBaseAdmin();
admin.createTable(desc, startKey, endKey, expectedRegions); 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(); regions = ht2.getRegionLocations();
assertEquals("Tried to create " + expectedRegions + " regions " + assertEquals("Tried to create " + expectedRegions + " regions " +
"but only found " + regions.size(), "but only found " + regions.size(),
@ -766,11 +766,11 @@ public class TestAdmin1 {
desc = new HTableDescriptor(TABLE_3); desc = new HTableDescriptor(TABLE_3);
desc.addFamily(new HColumnDescriptor(HConstants.CATALOG_FAMILY)); desc.addFamily(new HColumnDescriptor(HConstants.CATALOG_FAMILY));
admin = new HBaseAdmin(TEST_UTIL.getConfiguration()); admin = TEST_UTIL.getHBaseAdmin();
admin.createTable(desc, startKey, endKey, expectedRegions); 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(); regions = ht3.getRegionLocations();
assertEquals("Tried to create " + expectedRegions + " regions " + assertEquals("Tried to create " + expectedRegions + " regions " +
"but only found " + regions.size(), "but only found " + regions.size(),
@ -792,15 +792,13 @@ public class TestAdmin1 {
TableName TABLE_4 = TableName.valueOf(tableName.getNameAsString() + "_4"); TableName TABLE_4 = TableName.valueOf(tableName.getNameAsString() + "_4");
desc = new HTableDescriptor(TABLE_4); desc = new HTableDescriptor(TABLE_4);
desc.addFamily(new HColumnDescriptor(HConstants.CATALOG_FAMILY)); desc.addFamily(new HColumnDescriptor(HConstants.CATALOG_FAMILY));
Admin ladmin = new HBaseAdmin(TEST_UTIL.getConfiguration());
try { try {
ladmin.createTable(desc, splitKeys); admin.createTable(desc, splitKeys);
assertTrue("Should not be able to create this table because of " + assertTrue("Should not be able to create this table because of " +
"duplicate split keys", false); "duplicate split keys", false);
} catch(IllegalArgumentException iae) { } catch(IllegalArgumentException iae) {
// Expected // Expected
} }
ladmin.close();
} }
@Test (timeout=300000) @Test (timeout=300000)
@ -893,7 +891,7 @@ public class TestAdmin1 {
HTableDescriptor desc = new HTableDescriptor(tableName); HTableDescriptor desc = new HTableDescriptor(tableName);
desc.addFamily(new HColumnDescriptor(HConstants.CATALOG_FAMILY)); desc.addFamily(new HColumnDescriptor(HConstants.CATALOG_FAMILY));
admin.createTable(desc, splitKeys); admin.createTable(desc, splitKeys);
HTable ht = new HTable(TEST_UTIL.getConfiguration(), tableName); HTable ht = (HTable) TEST_UTIL.getConnection().getTable(tableName);
Map<HRegionInfo, ServerName> regions = ht.getRegionLocations(); Map<HRegionInfo, ServerName> regions = ht.getRegionLocations();
assertEquals("Tried to create " + expectedRegions + " regions " assertEquals("Tried to create " + expectedRegions + " regions "
+ "but only found " + regions.size(), expectedRegions, regions.size()); + "but only found " + regions.size(), expectedRegions, regions.size());
@ -1104,7 +1102,7 @@ public class TestAdmin1 {
Thread.sleep(10); Thread.sleep(10);
} while (oldRegions.size() != 9); //3 regions * 3 replicas } while (oldRegions.size() != 9); //3 regions * 3 replicas
// write some data to the table // write some data to the table
HTable ht = new HTable(TEST_UTIL.getConfiguration(), tableName); HTable ht = (HTable) TEST_UTIL.getConnection().getTable(tableName);
List<Put> puts = new ArrayList<Put>(); List<Put> puts = new ArrayList<Put>();
byte[] qualifier = "c".getBytes(); byte[] qualifier = "c".getBytes();
Put put = new Put(new byte[]{(byte)'1'}); Put put = new Put(new byte[]{(byte)'1'});
@ -1227,7 +1225,7 @@ public class TestAdmin1 {
} }
this.admin.disableTable(tableName); this.admin.disableTable(tableName);
try { try {
new HTable(TEST_UTIL.getConfiguration(), tableName); TEST_UTIL.getConnection().getTable(tableName);
} catch (org.apache.hadoop.hbase.DoNotRetryIOException e) { } catch (org.apache.hadoop.hbase.DoNotRetryIOException e) {
//expected //expected
} }

View File

@ -176,7 +176,7 @@ public class TestAdmin2 {
admin.createTable(htd1); admin.createTable(htd1);
admin.createTable(htd2); admin.createTable(htd2);
// Before fix, below would fail throwing a NoServerForRegionException. // 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 // Use 80 bit numbers to make sure we aren't limited
byte [] startKey = { 1, 1, 1, 1, 1, 1, 1, 1, 1, 1 }; byte [] startKey = { 1, 1, 1, 1, 1, 1, 1, 1, 1, 1 };
byte [] endKey = { 9, 9, 9, 9, 9, 9, 9, 9, 9, 9 }; 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)); HTableDescriptor htd = new HTableDescriptor(TableName.valueOf(name));
htd.addFamily(new HColumnDescriptor(HConstants.CATALOG_FAMILY)); htd.addFamily(new HColumnDescriptor(HConstants.CATALOG_FAMILY));
hbaseadmin.createTable(htd, startKey, endKey, expectedRegions); hbaseadmin.createTable(htd, startKey, endKey, expectedRegions);
hbaseadmin.close();
} finally { } finally {
TEST_UTIL.getConfiguration().setInt(HConstants.HBASE_RPC_TIMEOUT_KEY, oldTimeout); TEST_UTIL.getConfiguration().setInt(HConstants.HBASE_RPC_TIMEOUT_KEY, oldTimeout);
} }
@ -299,7 +298,7 @@ public class TestAdmin2 {
public void testTableNotFoundExceptionWithoutAnyTables() throws IOException { public void testTableNotFoundExceptionWithoutAnyTables() throws IOException {
TableName tableName = TableName TableName tableName = TableName
.valueOf("testTableNotFoundExceptionWithoutAnyTables"); .valueOf("testTableNotFoundExceptionWithoutAnyTables");
Table ht = new HTable(TEST_UTIL.getConfiguration(), tableName); Table ht = TEST_UTIL.getConnection().getTable(tableName);
ht.get(new Get("e".getBytes())); ht.get(new Get("e".getBytes()));
} }
@ -466,9 +465,7 @@ public class TestAdmin2 {
} }
private HBaseAdmin createTable(byte[] TABLENAME) throws IOException { private HBaseAdmin createTable(byte[] TABLENAME) throws IOException {
HBaseAdmin admin = TEST_UTIL.getHBaseAdmin();
Configuration config = TEST_UTIL.getConfiguration();
HBaseAdmin admin = new HBaseAdmin(config);
HTableDescriptor htd = new HTableDescriptor(TableName.valueOf(TABLENAME)); HTableDescriptor htd = new HTableDescriptor(TableName.valueOf(TABLENAME));
HColumnDescriptor hcd = new HColumnDescriptor("value"); HColumnDescriptor hcd = new HColumnDescriptor("value");
@ -608,14 +605,13 @@ public class TestAdmin2 {
private HRegionServer startAndWriteData(TableName tableName, byte[] value) private HRegionServer startAndWriteData(TableName tableName, byte[] value)
throws IOException, InterruptedException { throws IOException, InterruptedException {
// When the hbase:meta table can be opened, the region servers are running // When the hbase:meta table can be opened, the region servers are running
new HTable( TEST_UTIL.getConnection().getTable(TableName.META_TABLE_NAME).close();
TEST_UTIL.getConfiguration(), TableName.META_TABLE_NAME).close();
// Create the test table and open it // Create the test table and open it
HTableDescriptor desc = new HTableDescriptor(tableName); HTableDescriptor desc = new HTableDescriptor(tableName);
desc.addFamily(new HColumnDescriptor(HConstants.CATALOG_FAMILY)); desc.addFamily(new HColumnDescriptor(HConstants.CATALOG_FAMILY));
admin.createTable(desc); admin.createTable(desc);
Table table = new HTable(TEST_UTIL.getConfiguration(), tableName); Table table = TEST_UTIL.getConnection().getTable(tableName);
HRegionServer regionServer = TEST_UTIL.getRSForFirstRegionInTable(tableName); HRegionServer regionServer = TEST_UTIL.getRSForFirstRegionInTable(tableName);
for (int i = 1; i <= 256; i++) { // 256 writes should cause 8 log rolls 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 { public void testGetRegion() throws Exception {
// We use actual HBaseAdmin instance instead of going via Admin interface in // We use actual HBaseAdmin instance instead of going via Admin interface in
// here because makes use of an internal HBA method (TODO: Fix.). // 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"); final TableName tableName = TableName.valueOf("testGetRegion");
LOG.info("Started " + tableName); LOG.info("Started " + tableName);

View File

@ -82,9 +82,8 @@ public class TestClientOperationInterrupt {
} }
admin.deleteTable(tableName); 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); Put p = new Put(row1);
p.add(dummy, dummy, dummy); p.add(dummy, dummy, dummy);
ht.put(p); ht.put(p);
@ -106,7 +105,7 @@ public class TestClientOperationInterrupt {
@Override @Override
public void run() { public void run() {
try { try {
Table ht = new HTable(conf, tableName); Table ht = util.getConnection().getTable(tableName);
Result r = ht.get(new Get(row1)); Result r = ht.get(new Get(row1));
noEx.incrementAndGet(); noEx.incrementAndGet();
} catch (IOException e) { } catch (IOException e) {
@ -155,7 +154,7 @@ public class TestClientOperationInterrupt {
Thread.sleep(1); Thread.sleep(1);
} }
Table ht = new HTable(conf, tableName); Table ht = util.getConnection().getTable(tableName);
Result r = ht.get(new Get(row1)); Result r = ht.get(new Get(row1));
Assert.assertFalse(r.isEmpty()); Assert.assertFalse(r.isEmpty());
} }

View File

@ -98,12 +98,11 @@ public class TestClientTimeouts {
// Ensure the HBaseAdmin uses a new connection by changing Configuration. // Ensure the HBaseAdmin uses a new connection by changing Configuration.
Configuration conf = HBaseConfiguration.create(TEST_UTIL.getConfiguration()); Configuration conf = HBaseConfiguration.create(TEST_UTIL.getConfiguration());
conf.set(HConstants.HBASE_CLIENT_INSTANCE_ID, String.valueOf(-1)); conf.set(HConstants.HBASE_CLIENT_INSTANCE_ID, String.valueOf(-1));
HBaseAdmin admin = null; Admin admin = null;
Connection connection = null;
try { try {
admin = new HBaseAdmin(conf); connection = ConnectionFactory.createConnection(conf);
Connection connection = admin.getConnection(); admin = connection.getAdmin();
assertFalse(connection == lastConnection);
lastConnection = connection;
// run some admin commands // run some admin commands
HBaseAdmin.checkHBaseAvailable(conf); HBaseAdmin.checkHBaseAvailable(conf);
admin.setBalancerRunning(false, false); admin.setBalancerRunning(false, false);
@ -112,12 +111,17 @@ public class TestClientTimeouts {
// a MasterNotRunningException. It's a bug if we get other exceptions. // a MasterNotRunningException. It's a bug if we get other exceptions.
lastFailed = true; lastFailed = true;
} finally { } finally {
if(admin != null) {
admin.close(); admin.close();
if (admin.getConnection().isClosed()) { if (admin.getConnection().isClosed()) {
rpcClient = (RandomTimeoutRpcClient) RpcClientFactory rpcClient = (RandomTimeoutRpcClient) RpcClientFactory
.createClient(TEST_UTIL.getConfiguration(), TEST_UTIL.getClusterKey()); .createClient(TEST_UTIL.getConfiguration(), TEST_UTIL.getClusterKey());
} }
} }
if(connection != null) {
connection.close();
}
}
} }
// Ensure the RandomTimeoutRpcEngine is actually being used. // Ensure the RandomTimeoutRpcEngine is actually being used.
assertFalse(lastFailed); assertFalse(lastFailed);

View File

@ -100,7 +100,7 @@ public class TestCloneSnapshotFromClient {
// take an empty snapshot // take an empty snapshot
admin.snapshot(emptySnapshot, tableName); admin.snapshot(emptySnapshot, tableName);
HTable table = new HTable(TEST_UTIL.getConfiguration(), tableName); Table table = TEST_UTIL.getConnection().getTable(tableName);
try { try {
// enable table and insert data // enable table and insert data
admin.enableTable(tableName); admin.enableTable(tableName);

View File

@ -182,8 +182,7 @@ public class TestFromClientSide {
HTableDescriptor desc = new HTableDescriptor(TABLENAME); HTableDescriptor desc = new HTableDescriptor(TABLENAME);
desc.addFamily(hcd); desc.addFamily(hcd);
TEST_UTIL.getHBaseAdmin().createTable(desc); TEST_UTIL.getHBaseAdmin().createTable(desc);
Configuration c = TEST_UTIL.getConfiguration(); Table h = TEST_UTIL.getConnection().getTable(TABLENAME);
Table h = new HTable(c, TABLENAME);
long ts = System.currentTimeMillis(); long ts = System.currentTimeMillis();
Put p = new Put(T1, ts); Put p = new Put(T1, ts);
@ -415,7 +414,7 @@ public class TestFromClientSide {
putRows(ht, 3, value2, keyPrefix1); putRows(ht, 3, value2, keyPrefix1);
putRows(ht, 3, value2, keyPrefix2); putRows(ht, 3, value2, keyPrefix2);
putRows(ht, 3, value2, keyPrefix3); 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); System.out.println("Checking values for key: " + keyPrefix1);
assertEquals("Got back incorrect number of rows from scan", 3, assertEquals("Got back incorrect number of rows from scan", 3,
getNumberOfRows(keyPrefix1, value2, table)); getNumberOfRows(keyPrefix1, value2, table));
@ -641,8 +640,8 @@ public class TestFromClientSide {
private Map<HRegionInfo, ServerName> splitTable(final HTable t) private Map<HRegionInfo, ServerName> splitTable(final HTable t)
throws IOException, InterruptedException { throws IOException, InterruptedException {
// Split this table in two. // Split this table in two.
HBaseAdmin admin = new HBaseAdmin(TEST_UTIL.getConfiguration()); HBaseAdmin admin = TEST_UTIL.getHBaseAdmin();
admin.split(t.getTableName()); admin.split(t.getName());
admin.close(); admin.close();
Map<HRegionInfo, ServerName> regions = waitOnSplit(t); Map<HRegionInfo, ServerName> regions = waitOnSplit(t);
assertTrue(regions.size() > 1); assertTrue(regions.size() > 1);
@ -1748,7 +1747,7 @@ public class TestFromClientSide {
@Test @Test
public void testDeleteFamilyVersion() throws Exception { public void testDeleteFamilyVersion() throws Exception {
HBaseAdmin admin = new HBaseAdmin(TEST_UTIL.getConfiguration()); HBaseAdmin admin = TEST_UTIL.getHBaseAdmin();
byte [] TABLE = Bytes.toBytes("testDeleteFamilyVersion"); byte [] TABLE = Bytes.toBytes("testDeleteFamilyVersion");
byte [][] QUALIFIERS = makeNAscii(QUALIFIER, 1); byte [][] QUALIFIERS = makeNAscii(QUALIFIER, 1);
@ -1793,7 +1792,7 @@ public class TestFromClientSide {
byte [][] VALUES = makeN(VALUE, 5); byte [][] VALUES = makeN(VALUE, 5);
long [] ts = {1000, 2000, 3000, 4000, 5000}; 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); Table ht = TEST_UTIL.createTable(TABLE, FAMILY, 5);
Put put = null; Put put = null;
Result result = null; Result result = null;
@ -3633,7 +3632,7 @@ public class TestFromClientSide {
TableName TABLE = TableName.valueOf("testUpdatesWithMajorCompaction"); TableName TABLE = TableName.valueOf("testUpdatesWithMajorCompaction");
Table hTable = TEST_UTIL.createTable(TABLE, FAMILY, 10); 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 // Write a column with values at timestamp 1, 2 and 3
byte[] row = Bytes.toBytes("row2"); byte[] row = Bytes.toBytes("row2");
@ -3695,7 +3694,7 @@ public class TestFromClientSide {
String tableName = "testMajorCompactionBetweenTwoUpdates"; String tableName = "testMajorCompactionBetweenTwoUpdates";
byte [] TABLE = Bytes.toBytes(tableName); byte [] TABLE = Bytes.toBytes(tableName);
Table hTable = TEST_UTIL.createTable(TABLE, FAMILY, 10); 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 // Write a column with values at timestamp 1, 2 and 3
byte[] row = Bytes.toBytes("row3"); byte[] row = Bytes.toBytes("row3");
@ -4104,7 +4103,7 @@ public class TestFromClientSide {
for (int i = 0; i < tables.length; i++) { for (int i = 0; i < tables.length; i++) {
TEST_UTIL.createTable(tables[i], FAMILY); TEST_UTIL.createTable(tables[i], FAMILY);
} }
Admin admin = new HBaseAdmin(TEST_UTIL.getConfiguration()); Admin admin = TEST_UTIL.getHBaseAdmin();
HTableDescriptor[] ts = admin.listTables(); HTableDescriptor[] ts = admin.listTables();
HashSet<HTableDescriptor> result = new HashSet<HTableDescriptor>(ts.length); HashSet<HTableDescriptor> result = new HashSet<HTableDescriptor>(ts.length);
Collections.addAll(result, ts); Collections.addAll(result, ts);
@ -4196,7 +4195,7 @@ public class TestFromClientSide {
a.put(put); a.put(put);
// open a new connection to A and a connection to b // 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 // copy data from A to B
Scan scan = new Scan(); Scan scan = new Scan();
@ -4216,7 +4215,7 @@ public class TestFromClientSide {
} }
// Opening a new connection to A will cause the tables to be reloaded // 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 get = new Get(ROW);
get.addFamily(HConstants.CATALOG_FAMILY); get.addFamily(HConstants.CATALOG_FAMILY);
anotherA.get(get); anotherA.get(get);
@ -4226,7 +4225,7 @@ public class TestFromClientSide {
// to be reloaded. // to be reloaded.
// Test user metadata // Test user metadata
Admin admin = new HBaseAdmin(TEST_UTIL.getConfiguration()); Admin admin = TEST_UTIL.getHBaseAdmin();
// make a modifiable descriptor // make a modifiable descriptor
HTableDescriptor desc = new HTableDescriptor(a.getTableDescriptor()); HTableDescriptor desc = new HTableDescriptor(a.getTableDescriptor());
// offline the table // offline the table
@ -4984,12 +4983,9 @@ public class TestFromClientSide {
public void testScanMetrics() throws Exception { public void testScanMetrics() throws Exception {
TableName TABLENAME = TableName.valueOf("testScanMetrics"); TableName TABLENAME = TableName.valueOf("testScanMetrics");
Configuration conf = TEST_UTIL.getConfiguration();
TEST_UTIL.createTable(TABLENAME, FAMILY);
// Set up test table: // Set up test table:
// Create table: // Create table:
HTable ht = new HTable(conf, TABLENAME); HTable ht = TEST_UTIL.createTable(TABLENAME, FAMILY);
// Create multiple regions for this table // Create multiple regions for this table
int numOfRegions = TEST_UTIL.createMultiRegions(ht, FAMILY); int numOfRegions = TEST_UTIL.createMultiRegions(ht, FAMILY);
@ -5203,7 +5199,7 @@ public class TestFromClientSide {
byte [] family1 = Bytes.toBytes("f1"); byte [] family1 = Bytes.toBytes("f1");
byte [] family2 = Bytes.toBytes("f2"); byte [] family2 = Bytes.toBytes("f2");
try (HTable table = TEST_UTIL.createTable(TABLE, new byte[][] {family1, family2}, 10); 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 <HRegionInfo, ServerName> regionsMap = table.getRegionLocations(); Map <HRegionInfo, ServerName> regionsMap = table.getRegionLocations();
assertEquals(1, regionsMap.size()); assertEquals(1, regionsMap.size());
HRegionInfo regionInfo = regionsMap.keySet().iterator().next(); HRegionInfo regionInfo = regionsMap.keySet().iterator().next();

View File

@ -126,7 +126,7 @@ public class TestFromClientSide3 {
int sfCount = sf.size(); int sfCount = sf.size();
// TODO: replace this api with a synchronous flush after HBASE-2949 // 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) // synchronously poll wait for a new storefile to appear (flush happened)
while (ProtobufUtil.getStoreFiles( while (ProtobufUtil.getStoreFiles(
@ -408,7 +408,7 @@ public class TestFromClientSide3 {
HTableDescriptor desc = new HTableDescriptor(TableName.valueOf(Bytes.toBytes("test"))); HTableDescriptor desc = new HTableDescriptor(TableName.valueOf(Bytes.toBytes("test")));
desc.addFamily(new HColumnDescriptor(FAMILY)); desc.addFamily(new HColumnDescriptor(FAMILY));
admin.createTable(desc); 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 put = new Put(ROW_BYTES);
put.add(FAMILY, COL_QUAL, VAL_BYTES); put.add(FAMILY, COL_QUAL, VAL_BYTES);

View File

@ -349,7 +349,8 @@ public class TestHCM {
c2.setInt(RpcClient.FAILED_SERVER_EXPIRY_KEY, 0); // Server do not really expire c2.setInt(RpcClient.FAILED_SERVER_EXPIRY_KEY, 0); // Server do not really expire
c2.setBoolean(RpcClient.SPECIFIC_WRITE_THREAD, allowsInterrupt); 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 put = new Put(ROW);
put.add(FAM_NAM, ROW, ROW); put.add(FAM_NAM, ROW, ROW);
@ -409,6 +410,7 @@ public class TestHCM {
}); });
table.close(); table.close();
connection.close();
Assert.assertTrue("Unexpected exception is " + failed.get(), failed.get() == null); Assert.assertTrue("Unexpected exception is " + failed.get(), failed.get() == null);
TEST_UTIL.getHBaseAdmin().setBalancerRunning(previousBalance, true); 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(HConstants.HBASE_CLIENT_RETRIES_NUMBER, 1); // Don't retry: retry = test failed
c2.setInt(RpcClient.IDLE_TIME, idleTime); 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 put = new Put(ROW);
put.add(FAM_NAM, ROW, ROW); put.add(FAM_NAM, ROW, ROW);
@ -465,6 +468,8 @@ public class TestHCM {
LOG.info("we're done - time will change back"); LOG.info("we're done - time will change back");
table.close(); table.close();
connection.close();
EnvironmentEdgeManager.reset(); EnvironmentEdgeManager.reset();
TEST_UTIL.getHBaseAdmin().setBalancerRunning(previousBalance, true); TEST_UTIL.getHBaseAdmin().setBalancerRunning(previousBalance, true);
} }
@ -492,7 +497,8 @@ public class TestHCM {
c2.setInt(HConstants.HBASE_CLIENT_RETRIES_NUMBER, 1); c2.setInt(HConstants.HBASE_CLIENT_RETRIES_NUMBER, 1);
c2.setInt(HConstants.HBASE_RPC_TIMEOUT_KEY, 30 * 1000); 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); Put p = new Put(FAM_NAM);
p.add(FAM_NAM, FAM_NAM, FAM_NAM); p.add(FAM_NAM, FAM_NAM, FAM_NAM);
@ -538,6 +544,7 @@ public class TestHCM {
} }
table.close(); table.close();
connection.close();
} }
protected static final AtomicBoolean syncBlockingFilter = new AtomicBoolean(false); protected static final AtomicBoolean syncBlockingFilter = new AtomicBoolean(false);
@ -597,7 +604,8 @@ public class TestHCM {
TEST_UTIL.createTable(TABLE_NAME, FAM_NAM).close(); TEST_UTIL.createTable(TABLE_NAME, FAM_NAM).close();
Configuration conf = new Configuration(TEST_UTIL.getConfiguration()); Configuration conf = new Configuration(TEST_UTIL.getConfiguration());
conf.setInt(HConstants.HBASE_CLIENT_RETRIES_NUMBER, 1); 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.createMultiRegions(table, FAM_NAM);
TEST_UTIL.waitUntilAllRegionsAssigned(table.getName()); TEST_UTIL.waitUntilAllRegionsAssigned(table.getName());
@ -773,6 +781,7 @@ public class TestHCM {
TEST_UTIL.getConfiguration().setInt(HConstants.HBASE_CLIENT_RETRIES_NUMBER, TEST_UTIL.getConfiguration().setInt(HConstants.HBASE_CLIENT_RETRIES_NUMBER,
HConstants.DEFAULT_HBASE_CLIENT_RETRIES_NUMBER); HConstants.DEFAULT_HBASE_CLIENT_RETRIES_NUMBER);
table.close(); table.close();
connection.close();
} }
/** /**
@ -1240,7 +1249,7 @@ public class TestHCM {
try { try {
c1 = ConnectionManager.getConnectionInternal(config); c1 = ConnectionManager.getConnectionInternal(config);
LOG.info("HTable connection " + i + " " + c1); LOG.info("HTable connection " + i + " " + c1);
Table table = new HTable(config, TABLE_NAME4, pool); Table table = c1.getTable(TABLE_NAME4, pool);
table.close(); table.close();
LOG.info("HTable connection " + i + " closed " + c1); LOG.info("HTable connection " + i + " closed " + c1);
} catch (Exception e) { } catch (Exception e) {

View File

@ -170,7 +170,7 @@ public class TestMultiParallel {
@Test(timeout=300000) @Test(timeout=300000)
public void testBatchWithGet() throws Exception { public void testBatchWithGet() throws Exception {
LOG.info("test=testBatchWithGet"); LOG.info("test=testBatchWithGet");
Table table = new HTable(UTIL.getConfiguration(), TEST_TABLE); Table table = UTIL.getConnection().getTable(TEST_TABLE);
// load test data // load test data
List<Row> puts = constructPutRequests(); List<Row> puts = constructPutRequests();
@ -209,7 +209,7 @@ public class TestMultiParallel {
@Test @Test
public void testBadFam() throws Exception { public void testBadFam() throws Exception {
LOG.info("test=testBadFam"); LOG.info("test=testBadFam");
Table table = new HTable(UTIL.getConfiguration(), TEST_TABLE); Table table = UTIL.getConnection().getTable(TEST_TABLE);
List<Row> actions = new ArrayList<Row>(); List<Row> actions = new ArrayList<Row>();
Put p = new Put(Bytes.toBytes("row1")); Put p = new Put(Bytes.toBytes("row1"));
@ -262,7 +262,7 @@ public class TestMultiParallel {
private void doTestFlushCommits(boolean doAbort) throws Exception { private void doTestFlushCommits(boolean doAbort) throws Exception {
// Load the data // Load the data
LOG.info("get new table"); LOG.info("get new table");
HTable table = new HTable(UTIL.getConfiguration(), TEST_TABLE); Table table = UTIL.getConnection().getTable(TEST_TABLE);
table.setAutoFlushTo(false); table.setAutoFlushTo(false);
table.setWriteBufferSize(10 * 1024 * 1024); table.setWriteBufferSize(10 * 1024 * 1024);
@ -362,7 +362,7 @@ public class TestMultiParallel {
@Test(timeout=300000) @Test(timeout=300000)
public void testBatchWithDelete() throws Exception { public void testBatchWithDelete() throws Exception {
LOG.info("test=testBatchWithDelete"); LOG.info("test=testBatchWithDelete");
Table table = new HTable(UTIL.getConfiguration(), TEST_TABLE); Table table = UTIL.getConnection().getTable(TEST_TABLE);
// Load some data // Load some data
List<Row> puts = constructPutRequests(); List<Row> puts = constructPutRequests();
@ -391,7 +391,7 @@ public class TestMultiParallel {
@Test(timeout=300000) @Test(timeout=300000)
public void testHTableDeleteWithList() throws Exception { public void testHTableDeleteWithList() throws Exception {
LOG.info("test=testHTableDeleteWithList"); LOG.info("test=testHTableDeleteWithList");
Table table = new HTable(UTIL.getConfiguration(), TEST_TABLE); Table table = UTIL.getConnection().getTable(TEST_TABLE);
// Load some data // Load some data
List<Row> puts = constructPutRequests(); List<Row> puts = constructPutRequests();
@ -420,7 +420,7 @@ public class TestMultiParallel {
@Test(timeout=300000) @Test(timeout=300000)
public void testBatchWithManyColsInOneRowGetAndPut() throws Exception { public void testBatchWithManyColsInOneRowGetAndPut() throws Exception {
LOG.info("test=testBatchWithManyColsInOneRowGetAndPut"); LOG.info("test=testBatchWithManyColsInOneRowGetAndPut");
Table table = new HTable(UTIL.getConfiguration(), TEST_TABLE); Table table = UTIL.getConnection().getTable(TEST_TABLE);
List<Row> puts = new ArrayList<Row>(); List<Row> puts = new ArrayList<Row>();
for (int i = 0; i < 100; i++) { for (int i = 0; i < 100; i++) {
@ -461,7 +461,7 @@ public class TestMultiParallel {
final byte[] QUAL2 = Bytes.toBytes("qual2"); final byte[] QUAL2 = Bytes.toBytes("qual2");
final byte[] QUAL3 = Bytes.toBytes("qual3"); final byte[] QUAL3 = Bytes.toBytes("qual3");
final byte[] QUAL4 = Bytes.toBytes("qual4"); 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); Delete d = new Delete(ONE_ROW);
table.delete(d); table.delete(d);
Put put = new Put(ONE_ROW); Put put = new Put(ONE_ROW);
@ -583,7 +583,7 @@ public class TestMultiParallel {
@Test(timeout=300000) @Test(timeout=300000)
public void testBatchWithMixedActions() throws Exception { public void testBatchWithMixedActions() throws Exception {
LOG.info("test=testBatchWithMixedActions"); LOG.info("test=testBatchWithMixedActions");
Table table = new HTable(UTIL.getConfiguration(), TEST_TABLE); Table table = UTIL.getConnection().getTable(TEST_TABLE);
// Load some data to start // Load some data to start
Object[] results = table.batch(constructPutRequests()); Object[] results = table.batch(constructPutRequests());

View File

@ -214,7 +214,7 @@ public class TestReplicaWithCluster {
} }
HTU.getHBaseCluster().stopMaster(0); HTU.getHBaseCluster().stopMaster(0);
Admin admin = new HBaseAdmin(HTU.getConfiguration()); Admin admin = HTU.getHBaseAdmin();
nHdt =admin.getTableDescriptor(hdt.getTableName()); nHdt =admin.getTableDescriptor(hdt.getTableName());
Assert.assertEquals("fams=" + Arrays.toString(nHdt.getColumnFamilies()), Assert.assertEquals("fams=" + Arrays.toString(nHdt.getColumnFamilies()),
bHdt.getColumnFamilies().length + 1, nHdt.getColumnFamilies().length); bHdt.getColumnFamilies().length + 1, nHdt.getColumnFamilies().length);
@ -255,7 +255,7 @@ public class TestReplicaWithCluster {
Put p = new Put(row); Put p = new Put(row);
p.add(row, row, 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); table.put(p);
HTU.getHBaseAdmin().flush(table.getName()); HTU.getHBaseAdmin().flush(table.getName());
@ -279,7 +279,7 @@ public class TestReplicaWithCluster {
table.close(); table.close();
LOG.info("stale get on the first cluster done. Now for the second."); 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<Exception>() { Waiter.waitFor(HTU.getConfiguration(), 1000, new Waiter.Predicate<Exception>() {
@Override @Override
public boolean evaluate() throws Exception { public boolean evaluate() throws Exception {

View File

@ -111,7 +111,7 @@ public class TestRestoreSnapshotFromClient {
// take an empty snapshot // take an empty snapshot
admin.snapshot(emptySnapshot, tableName); admin.snapshot(emptySnapshot, tableName);
HTable table = new HTable(TEST_UTIL.getConfiguration(), tableName); Table table = TEST_UTIL.getConnection().getTable(tableName);
// enable table and insert data // enable table and insert data
admin.enableTable(tableName); admin.enableTable(tableName);
SnapshotTestingUtils.loadData(TEST_UTIL, table, 500, FAMILY); SnapshotTestingUtils.loadData(TEST_UTIL, table, 500, FAMILY);
@ -175,7 +175,7 @@ public class TestRestoreSnapshotFromClient {
public void testRestoreSchemaChange() throws Exception { public void testRestoreSchemaChange() throws Exception {
byte[] TEST_FAMILY2 = Bytes.toBytes("cf2"); 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 // Add one column family and put some data in it
admin.disableTable(tableName); admin.disableTable(tableName);

View File

@ -131,7 +131,8 @@ public class TestRpcControllerFactory {
// change one of the connection properties so we get a new HConnection with our configuration // 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); 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); table.setAutoFlushTo(false);
byte[] row = Bytes.toBytes("row"); byte[] row = Bytes.toBytes("row");
Put p = new Put(row); Put p = new Put(row);
@ -187,6 +188,7 @@ public class TestRpcControllerFactory {
counter = doScan(table, scanInfo, counter); counter = doScan(table, scanInfo, counter);
table.close(); table.close();
connection.close();
} }
int doScan(Table table, Scan scan, int expectedCount) throws IOException { int doScan(Table table, Scan scan, int expectedCount) throws IOException {

View File

@ -101,8 +101,7 @@ public class TestScannerTimeout {
LOG.info("START ************ test2481"); LOG.info("START ************ test2481");
Scan scan = new Scan(); Scan scan = new Scan();
scan.setCaching(1); scan.setCaching(1);
Table table = Table table = TEST_UTIL.getConnection().getTable(TABLE_NAME);
new HTable(new Configuration(TEST_UTIL.getConfiguration()), TABLE_NAME);
ResultScanner r = table.getScanner(scan); ResultScanner r = table.getScanner(scan);
int count = 0; int count = 0;
try { try {
@ -141,7 +140,9 @@ public class TestScannerTimeout {
// this new table // this new table
Configuration conf = new Configuration(TEST_UTIL.getConfiguration()); Configuration conf = new Configuration(TEST_UTIL.getConfiguration());
conf.setInt(HConstants.HBASE_CLIENT_SCANNER_TIMEOUT_PERIOD, SCANNER_TIMEOUT * 100); 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); ResultScanner r = higherScanTimeoutTable.getScanner(scan);
// This takes way less than SCANNER_TIMEOUT*100 // This takes way less than SCANNER_TIMEOUT*100
rs.abort("die!"); rs.abort("die!");
@ -149,6 +150,7 @@ public class TestScannerTimeout {
assertEquals(NB_ROWS, results.length); assertEquals(NB_ROWS, results.length);
r.close(); r.close();
higherScanTimeoutTable.close(); higherScanTimeoutTable.close();
connection.close();
LOG.info("END ************ test2772"); LOG.info("END ************ test2772");
} }
@ -175,7 +177,8 @@ public class TestScannerTimeout {
Configuration conf = new Configuration(TEST_UTIL.getConfiguration()); Configuration conf = new Configuration(TEST_UTIL.getConfiguration());
conf.setInt( conf.setInt(
HConstants.HBASE_CLIENT_SCANNER_TIMEOUT_PERIOD, SCANNER_TIMEOUT*100); 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"); LOG.info("START ************ TEST3686A---22");
ResultScanner r = table.getScanner(scan); ResultScanner r = table.getScanner(scan);
@ -193,6 +196,7 @@ public class TestScannerTimeout {
assertEquals(NB_ROWS, count); assertEquals(NB_ROWS, count);
r.close(); r.close();
table.close(); table.close();
connection.close();
LOG.info("************ END TEST3686A"); LOG.info("************ END TEST3686A");
} }
@ -214,7 +218,8 @@ public class TestScannerTimeout {
// this new table // this new table
Configuration conf = new Configuration(TEST_UTIL.getConfiguration()); Configuration conf = new Configuration(TEST_UTIL.getConfiguration());
conf.setInt(HConstants.HBASE_CLIENT_SCANNER_TIMEOUT_PERIOD, SCANNER_TIMEOUT * 100); 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); ResultScanner r = higherScanTimeoutTable.getScanner(scan);
int count = 1; int count = 1;
r.next(); r.next();
@ -226,6 +231,7 @@ public class TestScannerTimeout {
assertEquals(NB_ROWS, count); assertEquals(NB_ROWS, count);
r.close(); r.close();
higherScanTimeoutTable.close(); higherScanTimeoutTable.close();
connection.close();
LOG.info("END ************ END test3686b"); LOG.info("END ************ END test3686b");
} }

View File

@ -18,19 +18,17 @@
package org.apache.hadoop.hbase.client; package org.apache.hadoop.hbase.client;
import java.util.List;
import org.apache.commons.logging.Log; import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory; import org.apache.commons.logging.LogFactory;
import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.fs.FileSystem; import org.apache.hadoop.fs.FileSystem;
import org.apache.hadoop.fs.Path; import org.apache.hadoop.fs.Path;
import org.apache.hadoop.hbase.TableName;
import org.apache.hadoop.hbase.HBaseTestingUtility; import org.apache.hadoop.hbase.HBaseTestingUtility;
import org.apache.hadoop.hbase.HColumnDescriptor; import org.apache.hadoop.hbase.HColumnDescriptor;
import org.apache.hadoop.hbase.HConstants; import org.apache.hadoop.hbase.HConstants;
import org.apache.hadoop.hbase.HRegionInfo; import org.apache.hadoop.hbase.HRegionInfo;
import org.apache.hadoop.hbase.HTableDescriptor; 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.master.snapshot.SnapshotManager;
import org.apache.hadoop.hbase.regionserver.ConstantSizeRegionSplitPolicy; import org.apache.hadoop.hbase.regionserver.ConstantSizeRegionSplitPolicy;
import org.apache.hadoop.hbase.snapshot.SnapshotTestingUtils; import org.apache.hadoop.hbase.snapshot.SnapshotTestingUtils;
@ -45,6 +43,8 @@ import org.junit.BeforeClass;
import org.junit.Test; import org.junit.Test;
import org.junit.experimental.categories.Category; 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 * 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); TableName cloneTableName = TableName.valueOf("test-clone-" + localTableName);
admin.cloneSnapshot(snapshotName, cloneTableName); 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); final int clonedTableRowCount = UTIL.countRows(clonedTable);
Assert.assertEquals( Assert.assertEquals(

View File

@ -149,7 +149,7 @@ public class TestSnapshotFromClient {
SnapshotTestingUtils.assertNoSnapshots(admin); SnapshotTestingUtils.assertNoSnapshots(admin);
// put some stuff in the table // 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); UTIL.loadTable(table, TEST_FAM);
table.close(); table.close();
@ -185,7 +185,7 @@ public class TestSnapshotFromClient {
SnapshotTestingUtils.assertNoSnapshots(admin); SnapshotTestingUtils.assertNoSnapshots(admin);
// put some stuff in the table // 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); UTIL.loadTable(table, TEST_FAM, false);
LOG.debug("FS state before disable:"); LOG.debug("FS state before disable:");

View File

@ -171,7 +171,7 @@ public class TestSnapshotMetadata {
assertTrue(htd.getConfiguration().size() > 0); assertTrue(htd.getConfiguration().size() > 0);
admin.createTable(htd); admin.createTable(htd);
Table original = new HTable(UTIL.getConfiguration(), originalTableName); Table original = UTIL.getConnection().getTable(originalTableName);
originalTableName = TableName.valueOf(sourceTableNameAsString); originalTableName = TableName.valueOf(sourceTableNameAsString);
originalTableDescriptor = admin.getTableDescriptor(originalTableName); originalTableDescriptor = admin.getTableDescriptor(originalTableName);
originalTableDescription = originalTableDescriptor.toStringCustomizedValues(); originalTableDescription = originalTableDescriptor.toStringCustomizedValues();
@ -201,7 +201,7 @@ public class TestSnapshotMetadata {
familiesList, snapshotNameAsString, rootDir, fs, /* onlineSnapshot= */ false); familiesList, snapshotNameAsString, rootDir, fs, /* onlineSnapshot= */ false);
admin.cloneSnapshot(snapshotName, clonedTableName); admin.cloneSnapshot(snapshotName, clonedTableName);
Table clonedTable = new HTable(UTIL.getConfiguration(), clonedTableName); Table clonedTable = UTIL.getConnection().getTable(clonedTableName);
HTableDescriptor cloneHtd = admin.getTableDescriptor(clonedTableName); HTableDescriptor cloneHtd = admin.getTableDescriptor(clonedTableName);
assertEquals( assertEquals(
originalTableDescription.replace(originalTableName.getNameAsString(),clonedTableNameAsString), originalTableDescription.replace(originalTableName.getNameAsString(),clonedTableNameAsString),
@ -265,7 +265,7 @@ public class TestSnapshotMetadata {
List<byte[]> familiesWithDataList = new ArrayList<byte[]>(); List<byte[]> familiesWithDataList = new ArrayList<byte[]>();
List<byte[]> emptyFamiliesList = new ArrayList<byte[]>(); List<byte[]> emptyFamiliesList = new ArrayList<byte[]>();
if (addData) { if (addData) {
HTable original = new HTable(UTIL.getConfiguration(), originalTableName); Table original = UTIL.getConnection().getTable(originalTableName);
UTIL.loadTable(original, familyForUpdate); // family arbitrarily chosen UTIL.loadTable(original, familyForUpdate); // family arbitrarily chosen
original.close(); original.close();
@ -310,7 +310,7 @@ public class TestSnapshotMetadata {
admin.enableTable(originalTableName); admin.enableTable(originalTableName);
// verify that the descrption is reverted // verify that the descrption is reverted
Table original = new HTable(UTIL.getConfiguration(), originalTableName); Table original = UTIL.getConnection().getTable(originalTableName);
try { try {
assertTrue(originalTableDescriptor.equals(admin.getTableDescriptor(originalTableName))); assertTrue(originalTableDescriptor.equals(admin.getTableDescriptor(originalTableName)));
assertTrue(originalTableDescriptor.equals(original.getTableDescriptor())); assertTrue(originalTableDescriptor.equals(original.getTableDescriptor()));

View File

@ -92,7 +92,7 @@ public class TestTableSnapshotScanner {
Admin admin = util.getHBaseAdmin(); Admin admin = util.getHBaseAdmin();
// put some stuff in the table // put some stuff in the table
HTable table = new HTable(util.getConfiguration(), tableName); Table table = util.getConnection().getTable(tableName);
util.loadTable(table, FAMILIES); util.loadTable(table, FAMILIES);
Path rootDir = FSUtils.getRootDir(util.getConfiguration()); Path rootDir = FSUtils.getRootDir(util.getConfiguration());

View File

@ -82,7 +82,7 @@ public class TestConstraint {
Constraints.add(desc, CheckWasRunConstraint.class); Constraints.add(desc, CheckWasRunConstraint.class);
util.getHBaseAdmin().createTable(desc); util.getHBaseAdmin().createTable(desc);
Table table = new HTable(util.getConfiguration(), tableName); Table table = util.getConnection().getTable(tableName);
try { try {
// test that we don't fail on a valid put // test that we don't fail on a valid put
Put put = new Put(row1); Put put = new Put(row1);
@ -114,7 +114,7 @@ public class TestConstraint {
Constraints.add(desc, AllFailConstraint.class); Constraints.add(desc, AllFailConstraint.class);
util.getHBaseAdmin().createTable(desc); util.getHBaseAdmin().createTable(desc);
Table table = new HTable(util.getConfiguration(), tableName); Table table = util.getConnection().getTable(tableName);
// test that we do fail on violation // test that we do fail on violation
Put put = new Put(row1); Put put = new Put(row1);
@ -157,7 +157,7 @@ public class TestConstraint {
Constraints.disableConstraint(desc, AllFailConstraint.class); Constraints.disableConstraint(desc, AllFailConstraint.class);
util.getHBaseAdmin().createTable(desc); util.getHBaseAdmin().createTable(desc);
Table table = new HTable(util.getConfiguration(), tableName); Table table = util.getConnection().getTable(tableName);
try { try {
// test that we don't fail because its disabled // test that we don't fail because its disabled
Put put = new Put(row1); Put put = new Put(row1);
@ -189,7 +189,7 @@ public class TestConstraint {
Constraints.disable(desc); Constraints.disable(desc);
util.getHBaseAdmin().createTable(desc); util.getHBaseAdmin().createTable(desc);
Table table = new HTable(util.getConfiguration(), tableName); Table table = util.getConnection().getTable(tableName);
try { try {
// test that we do fail on violation // test that we do fail on violation
Put put = new Put(row1); Put put = new Put(row1);
@ -221,7 +221,7 @@ public class TestConstraint {
CheckWasRunConstraint.wasRun = false; CheckWasRunConstraint.wasRun = false;
util.getHBaseAdmin().createTable(desc); util.getHBaseAdmin().createTable(desc);
Table table = new HTable(util.getConfiguration(), tableName); Table table = util.getConnection().getTable(tableName);
// test that we do fail on violation // test that we do fail on violation
Put put = new Put(row1); Put put = new Put(row1);

View File

@ -26,6 +26,7 @@ import java.util.Map;
import java.util.TreeMap; import java.util.TreeMap;
import org.apache.hadoop.hbase.client.Admin; 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.client.Table;
import org.apache.hadoop.hbase.testclassification.CoprocessorTests; import org.apache.hadoop.hbase.testclassification.CoprocessorTests;
import org.apache.hadoop.hbase.testclassification.MediumTests; import org.apache.hadoop.hbase.testclassification.MediumTests;
@ -86,14 +87,14 @@ public class TestBatchCoprocessorEndpoint {
conf.setStrings(CoprocessorHost.MASTER_COPROCESSOR_CONF_KEY, conf.setStrings(CoprocessorHost.MASTER_COPROCESSOR_CONF_KEY,
ProtobufCoprocessorService.class.getName()); ProtobufCoprocessorService.class.getName());
util.startMiniCluster(2); util.startMiniCluster(2);
Admin admin = new HBaseAdmin(conf); Admin admin = util.getHBaseAdmin();
HTableDescriptor desc = new HTableDescriptor(TEST_TABLE); HTableDescriptor desc = new HTableDescriptor(TEST_TABLE);
desc.addFamily(new HColumnDescriptor(TEST_FAMILY)); desc.addFamily(new HColumnDescriptor(TEST_FAMILY));
admin.createTable(desc, new byte[][]{ROWS[rowSeperator1], ROWS[rowSeperator2]}); admin.createTable(desc, new byte[][]{ROWS[rowSeperator1], ROWS[rowSeperator2]});
util.waitUntilAllRegionsAssigned(TEST_TABLE); util.waitUntilAllRegionsAssigned(TEST_TABLE);
admin.close(); admin.close();
Table table = new HTable(conf, TEST_TABLE); Table table = util.getConnection().getTable(TEST_TABLE);
for (int i = 0; i < ROWSIZE; i++) { for (int i = 0; i < ROWSIZE; i++) {
Put put = new Put(ROWS[i]); Put put = new Put(ROWS[i]);
put.add(TEST_FAMILY, TEST_QUALIFIER, Bytes.toBytes(i)); put.add(TEST_FAMILY, TEST_QUALIFIER, Bytes.toBytes(i));
@ -109,7 +110,7 @@ public class TestBatchCoprocessorEndpoint {
@Test @Test
public void testAggregationNullResponse() throws Throwable { 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.Builder builder =
ColumnAggregationWithNullResponseProtos.SumRequest ColumnAggregationWithNullResponseProtos.SumRequest
.newBuilder(); .newBuilder();
@ -162,7 +163,7 @@ public class TestBatchCoprocessorEndpoint {
@Test @Test
public void testAggregationWithReturnValue() throws Throwable { public void testAggregationWithReturnValue() throws Throwable {
Table table = new HTable(util.getConfiguration(), TEST_TABLE); Table table = util.getConnection().getTable(TEST_TABLE);
Map<byte[], SumResponse> results = sum(table, TEST_FAMILY, TEST_QUALIFIER, ROWS[0], Map<byte[], SumResponse> results = sum(table, TEST_FAMILY, TEST_QUALIFIER, ROWS[0],
ROWS[ROWS.length - 1]); ROWS[ROWS.length - 1]);
int sumResult = 0; int sumResult = 0;
@ -198,7 +199,7 @@ public class TestBatchCoprocessorEndpoint {
@Test @Test
public void testAggregation() throws Throwable { public void testAggregation() throws Throwable {
Table table = new HTable(util.getConfiguration(), TEST_TABLE); Table table = util.getConnection().getTable(TEST_TABLE);
Map<byte[], SumResponse> results = sum(table, TEST_FAMILY, TEST_QUALIFIER, Map<byte[], SumResponse> results = sum(table, TEST_FAMILY, TEST_QUALIFIER,
ROWS[0], ROWS[ROWS.length - 1]); ROWS[0], ROWS[ROWS.length - 1]);
int sumResult = 0; int sumResult = 0;
@ -231,7 +232,7 @@ public class TestBatchCoprocessorEndpoint {
@Test @Test
public void testAggregationWithErrors() throws Throwable { public void testAggregationWithErrors() throws Throwable {
Table table = new HTable(util.getConfiguration(), TEST_TABLE); Table table = util.getConnection().getTable(TEST_TABLE);
final Map<byte[], ColumnAggregationWithErrorsProtos.SumResponse> results = final Map<byte[], ColumnAggregationWithErrorsProtos.SumResponse> results =
Collections.synchronizedMap( Collections.synchronizedMap(
new TreeMap<byte[], ColumnAggregationWithErrorsProtos.SumResponse>( new TreeMap<byte[], ColumnAggregationWithErrorsProtos.SumResponse>(

View File

@ -94,14 +94,14 @@ public class TestCoprocessorEndpoint {
conf.setStrings(CoprocessorHost.MASTER_COPROCESSOR_CONF_KEY, conf.setStrings(CoprocessorHost.MASTER_COPROCESSOR_CONF_KEY,
ProtobufCoprocessorService.class.getName()); ProtobufCoprocessorService.class.getName());
util.startMiniCluster(2); util.startMiniCluster(2);
Admin admin = new HBaseAdmin(conf); Admin admin = util.getHBaseAdmin();
HTableDescriptor desc = new HTableDescriptor(TEST_TABLE); HTableDescriptor desc = new HTableDescriptor(TEST_TABLE);
desc.addFamily(new HColumnDescriptor(TEST_FAMILY)); desc.addFamily(new HColumnDescriptor(TEST_FAMILY));
admin.createTable(desc, new byte[][]{ROWS[rowSeperator1], ROWS[rowSeperator2]}); admin.createTable(desc, new byte[][]{ROWS[rowSeperator1], ROWS[rowSeperator2]});
util.waitUntilAllRegionsAssigned(TEST_TABLE); util.waitUntilAllRegionsAssigned(TEST_TABLE);
admin.close(); admin.close();
Table table = new HTable(conf, TEST_TABLE); Table table = util.getConnection().getTable(TEST_TABLE);
for (int i = 0; i < ROWSIZE; i++) { for (int i = 0; i < ROWSIZE; i++) {
Put put = new Put(ROWS[i]); Put put = new Put(ROWS[i]);
put.add(TEST_FAMILY, TEST_QUALIFIER, Bytes.toBytes(i)); put.add(TEST_FAMILY, TEST_QUALIFIER, Bytes.toBytes(i));
@ -140,7 +140,7 @@ public class TestCoprocessorEndpoint {
@Test @Test
public void testAggregation() throws Throwable { public void testAggregation() throws Throwable {
Table table = new HTable(util.getConfiguration(), TEST_TABLE); Table table = util.getConnection().getTable(TEST_TABLE);
Map<byte[], Long> results = sum(table, TEST_FAMILY, TEST_QUALIFIER, Map<byte[], Long> results = sum(table, TEST_FAMILY, TEST_QUALIFIER,
ROWS[0], ROWS[ROWS.length-1]); ROWS[0], ROWS[ROWS.length-1]);
int sumResult = 0; int sumResult = 0;
@ -174,7 +174,7 @@ public class TestCoprocessorEndpoint {
@Test @Test
public void testCoprocessorService() throws Throwable { public void testCoprocessorService() throws Throwable {
HTable table = new HTable(util.getConfiguration(), TEST_TABLE); HTable table = (HTable) util.getConnection().getTable(TEST_TABLE);
NavigableMap<HRegionInfo,ServerName> regions = table.getRegionLocations(); NavigableMap<HRegionInfo,ServerName> regions = table.getRegionLocations();
final TestProtos.EchoRequestProto request = final TestProtos.EchoRequestProto request =
@ -248,7 +248,7 @@ public class TestCoprocessorEndpoint {
@Test @Test
public void testCoprocessorServiceNullResponse() throws Throwable { public void testCoprocessorServiceNullResponse() throws Throwable {
HTable table = new HTable(util.getConfiguration(), TEST_TABLE); HTable table = (HTable) util.getConnection().getTable(TEST_TABLE);
NavigableMap<HRegionInfo,ServerName> regions = table.getRegionLocations(); NavigableMap<HRegionInfo,ServerName> regions = table.getRegionLocations();
final TestProtos.EchoRequestProto request = final TestProtos.EchoRequestProto request =
@ -299,7 +299,7 @@ public class TestCoprocessorEndpoint {
Configuration configuration = new Configuration(util.getConfiguration()); Configuration configuration = new Configuration(util.getConfiguration());
// Make it not retry forever // Make it not retry forever
configuration.setInt(HConstants.HBASE_CLIENT_RETRIES_NUMBER, 1); configuration.setInt(HConstants.HBASE_CLIENT_RETRIES_NUMBER, 1);
Table table = new HTable(configuration, TEST_TABLE); Table table = util.getConnection().getTable(TEST_TABLE);
try { try {
CoprocessorRpcChannel protocol = table.coprocessorService(ROWS[0]); CoprocessorRpcChannel protocol = table.coprocessorService(ROWS[0]);

View File

@ -18,23 +18,12 @@
*/ */
package org.apache.hadoop.hbase.coprocessor; 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.HBaseTestingUtility;
import org.apache.hadoop.hbase.HColumnDescriptor; import org.apache.hadoop.hbase.HColumnDescriptor;
import org.apache.hadoop.hbase.HTableDescriptor; import org.apache.hadoop.hbase.HTableDescriptor;
import org.apache.hadoop.hbase.TableName; import org.apache.hadoop.hbase.TableName;
import org.apache.hadoop.hbase.client.Admin; import org.apache.hadoop.hbase.client.Admin;
import org.apache.hadoop.hbase.client.Durability; 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.Put;
import org.apache.hadoop.hbase.client.Result; import org.apache.hadoop.hbase.client.Result;
import org.apache.hadoop.hbase.client.ResultScanner; import org.apache.hadoop.hbase.client.ResultScanner;
@ -50,6 +39,16 @@ import org.junit.BeforeClass;
import org.junit.Test; import org.junit.Test;
import org.junit.experimental.categories.Category; 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. * 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(primary);
admin.createTable(other); 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' }); Put p = new Put(new byte[] { 'a' });
p.add(family, null, new byte[] { 'a' }); p.add(family, null, new byte[] { 'a' });
table.put(p); table.put(p);
table.close(); 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]); assertTrue("Didn't complete update to target table!", completeCheck[0]);
assertEquals("Didn't find inserted row", 1, getKeyValueCount(target)); assertEquals("Didn't find inserted row", 1, getKeyValueCount(target));
target.close(); target.close();

View File

@ -93,7 +93,7 @@ public class TestRegionObserverBypass {
*/ */
@Test @Test
public void testSimple() throws Exception { public void testSimple() throws Exception {
Table t = new HTable(util.getConfiguration(), tableName); Table t = util.getConnection().getTable(tableName);
Put p = new Put(row1); Put p = new Put(row1);
p.add(test,dummy,dummy); p.add(test,dummy,dummy);
// before HBASE-4331, this would throw an exception // 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 //previous deletes will eclipse successive puts having the same timestamp
EnvironmentEdgeManagerTestHelper.injectEdge(new IncrementingEnvironmentEdge()); EnvironmentEdgeManagerTestHelper.injectEdge(new IncrementingEnvironmentEdge());
Table t = new HTable(util.getConfiguration(), tableName); Table t = util.getConnection().getTable(tableName);
List<Put> puts = new ArrayList<Put>(); List<Put> puts = new ArrayList<Put>();
Put p = new Put(row1); Put p = new Put(row1);
p.add(dummy,dummy,dummy); p.add(dummy,dummy,dummy);

View File

@ -340,7 +340,7 @@ public class TestRegionObserverInterface {
new Boolean[] {false, false, false, false} 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 put = new Put(ROW);
put.add(A, A, A); put.add(A, A, A);
table.put(put); table.put(put);
@ -390,7 +390,7 @@ public class TestRegionObserverInterface {
new Boolean[] {false, false} new Boolean[] {false, false}
); );
Table table = new HTable(util.getConfiguration(), tableName); Table table = util.getConnection().getTable(tableName);
Put put = new Put(ROW); Put put = new Put(ROW);
put.add(A, A, A); put.add(A, A, A);
table.put(put); table.put(put);
@ -497,7 +497,7 @@ public class TestRegionObserverInterface {
htd.addCoprocessor(EvenOnlyCompactor.class.getName()); htd.addCoprocessor(EvenOnlyCompactor.class.getName());
admin.createTable(htd); admin.createTable(htd);
Table table = new HTable(util.getConfiguration(), compactTable); Table table = util.getConnection().getTable(compactTable);
for (long i=1; i<=10; i++) { for (long i=1; i<=10; i++) {
byte[] iBytes = Bytes.toBytes(i); byte[] iBytes = Bytes.toBytes(i);
Put put = new Put(iBytes); Put put = new Put(iBytes);

View File

@ -259,7 +259,7 @@ public class TestRegionObserverScannerOpenHook {
Admin admin = UTIL.getHBaseAdmin(); Admin admin = UTIL.getHBaseAdmin();
admin.createTable(desc); 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 a row and flush it to disk
Put put = new Put(ROW); Put put = new Put(ROW);

View File

@ -73,7 +73,7 @@ public class TestRegionServerCoprocessorEndpoint {
new BlockingRpcCallback<DummyRegionServerEndpointProtos.DummyResponse>(); new BlockingRpcCallback<DummyRegionServerEndpointProtos.DummyResponse>();
DummyRegionServerEndpointProtos.DummyService service = DummyRegionServerEndpointProtos.DummyService service =
ProtobufUtil.newServiceStub(DummyRegionServerEndpointProtos.DummyService.class, ProtobufUtil.newServiceStub(DummyRegionServerEndpointProtos.DummyService.class,
new HBaseAdmin(CONF).coprocessorService(serverName)); TEST_UTIL.getHBaseAdmin().coprocessorService(serverName));
service.dummyCall(controller, service.dummyCall(controller,
DummyRegionServerEndpointProtos.DummyRequest.getDefaultInstance(), rpcCallback); DummyRegionServerEndpointProtos.DummyRequest.getDefaultInstance(), rpcCallback);
assertEquals(DUMMY_VALUE, rpcCallback.get().getValue()); assertEquals(DUMMY_VALUE, rpcCallback.get().getValue());

View File

@ -76,7 +76,7 @@ public class TestRegionServerObserver {
// Start the cluster // Start the cluster
HBaseTestingUtility TEST_UTIL = new HBaseTestingUtility(conf); HBaseTestingUtility TEST_UTIL = new HBaseTestingUtility(conf);
TEST_UTIL.startMiniCluster(NUM_MASTERS, NUM_RS); TEST_UTIL.startMiniCluster(NUM_MASTERS, NUM_RS);
Admin admin = new HBaseAdmin(conf); Admin admin = TEST_UTIL.getHBaseAdmin();
try { try {
MiniHBaseCluster cluster = TEST_UTIL.getHBaseCluster(); MiniHBaseCluster cluster = TEST_UTIL.getHBaseCluster();
HRegionServer regionServer = cluster.getRegionServer(0); HRegionServer regionServer = cluster.getRegionServer(0);

View File

@ -37,6 +37,9 @@ import org.apache.hadoop.hbase.HTableDescriptor;
import org.apache.hadoop.hbase.MasterNotRunningException; import org.apache.hadoop.hbase.MasterNotRunningException;
import org.apache.hadoop.hbase.TableName; import org.apache.hadoop.hbase.TableName;
import org.apache.hadoop.hbase.ZooKeeperConnectionException; 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.HBaseAdmin;
import org.apache.hadoop.hbase.client.HTable; import org.apache.hadoop.hbase.client.HTable;
import org.apache.hadoop.hbase.client.ScannerCallable; import org.apache.hadoop.hbase.client.ScannerCallable;
@ -58,13 +61,13 @@ import org.junit.experimental.categories.Category;
@Category({FilterTests.class, MediumTests.class}) @Category({FilterTests.class, MediumTests.class})
public class FilterTestingCluster { public class FilterTestingCluster {
private static final HBaseTestingUtility TEST_UTIL = new HBaseTestingUtility(); private static final HBaseTestingUtility TEST_UTIL = new HBaseTestingUtility();
private static Configuration conf = null; private static Connection connection;
private static HBaseAdmin admin = null; private static Admin admin = null;
private static List<String> createdTables = new ArrayList<>(); private static List<TableName> 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); 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)); HColumnDescriptor colDef = new HColumnDescriptor(Bytes.toBytes(columnFamilyName));
desc.addFamily(colDef); desc.addFamily(colDef);
@ -77,15 +80,15 @@ public class FilterTestingCluster {
} }
} }
protected static Table openTable(String tableName) throws IOException { protected static Table openTable(TableName tableName) throws IOException {
Table table = new HTable(conf, tableName); Table table = connection.getTable(tableName);
assertTrue("Fail to create the table", admin.tableExists(tableName)); assertTrue("Fail to create the table", admin.tableExists(tableName));
return table; return table;
} }
private static void deleteTables() { private static void deleteTables() {
if (admin != null) { if (admin != null) {
for (String tableName: createdTables){ for (TableName tableName: createdTables){
try { try {
if (admin.tableExists(tableName)) { if (admin.tableExists(tableName)) {
admin.disableTable(tableName); admin.disableTable(tableName);
@ -99,10 +102,11 @@ public class FilterTestingCluster {
} }
private static void initialize(Configuration conf) { private static void initialize(Configuration conf) {
FilterTestingCluster.conf = HBaseConfiguration.create(conf); conf = HBaseConfiguration.create(conf);
FilterTestingCluster.conf.setInt(HConstants.HBASE_CLIENT_RETRIES_NUMBER, 1); conf.setInt(HConstants.HBASE_CLIENT_RETRIES_NUMBER, 1);
try { try {
admin = new HBaseAdmin(conf); connection = ConnectionFactory.createConnection(conf);
admin = connection.getAdmin();
} catch (MasterNotRunningException e) { } catch (MasterNotRunningException e) {
assertNull("Master is not running", e); assertNull("Master is not running", e);
} catch (ZooKeeperConnectionException e) { } catch (ZooKeeperConnectionException e) {
@ -124,7 +128,7 @@ public class FilterTestingCluster {
@AfterClass @AfterClass
public static void tearDown() throws Exception { public static void tearDown() throws Exception {
deleteTables(); deleteTables();
connection.close();
TEST_UTIL.shutdownMiniCluster(); TEST_UTIL.shutdownMiniCluster();
} }
} }

View File

@ -30,6 +30,7 @@ import java.util.List;
import org.apache.commons.logging.Log; import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory; import org.apache.commons.logging.LogFactory;
import org.apache.hadoop.hbase.Cell; 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.Put;
import org.apache.hadoop.hbase.client.Result; import org.apache.hadoop.hbase.client.Result;
import org.apache.hadoop.hbase.client.ResultScanner; import org.apache.hadoop.hbase.client.ResultScanner;
@ -50,7 +51,7 @@ public class TestFilterWithScanLimits extends FilterTestingCluster {
private static final Log LOG = LogFactory private static final Log LOG = LogFactory
.getLog(TestFilterWithScanLimits.class); .getLog(TestFilterWithScanLimits.class);
private static final String tableName = "scanWithLimit"; private static final TableName tableName = TableName.valueOf("scanWithLimit");
private static final String columnFamily = "f1"; private static final String columnFamily = "f1";
@Test @Test

View File

@ -36,6 +36,8 @@ import org.apache.hadoop.hbase.HTableDescriptor;
import org.apache.hadoop.hbase.TableName; import org.apache.hadoop.hbase.TableName;
import org.apache.hadoop.hbase.MasterNotRunningException; import org.apache.hadoop.hbase.MasterNotRunningException;
import org.apache.hadoop.hbase.ZooKeeperConnectionException; 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.HBaseAdmin;
import org.apache.hadoop.hbase.client.HTable; import org.apache.hadoop.hbase.client.HTable;
import org.apache.hadoop.hbase.client.Put; import org.apache.hadoop.hbase.client.Put;
@ -66,6 +68,7 @@ public class TestFilterWrapper {
private static Configuration conf = null; private static Configuration conf = null;
private static HBaseAdmin admin = null; private static HBaseAdmin admin = null;
private static TableName name = TableName.valueOf("test"); private static TableName name = TableName.valueOf("test");
private static Connection connection;
@Test @Test
public void testFilterWrapper() { public void testFilterWrapper() {
@ -84,7 +87,7 @@ public class TestFilterWrapper {
FilterList filter = new FilterList(fs); FilterList filter = new FilterList(fs);
scan.setFilter(filter); scan.setFilter(filter);
Table table = new HTable(conf, name); Table table = connection.getTable(name);
ResultScanner scanner = table.getScanner(scan); ResultScanner scanner = table.getScanner(scan);
// row2 (c1-c4) and row3(c1-c4) are returned // row2 (c1-c4) and row3(c1-c4) are returned
@ -111,7 +114,7 @@ public class TestFilterWrapper {
private static void prepareData() { private static void prepareData() {
try { try {
Table table = new HTable(TestFilterWrapper.conf, name); Table table = connection.getTable(name);
assertTrue("Fail to create the table", admin.tableExists(name)); assertTrue("Fail to create the table", admin.tableExists(name));
List<Put> puts = new ArrayList<Put>(); List<Put> puts = new ArrayList<Put>();
@ -173,7 +176,8 @@ public class TestFilterWrapper {
TestFilterWrapper.conf = HBaseConfiguration.create(conf); TestFilterWrapper.conf = HBaseConfiguration.create(conf);
TestFilterWrapper.conf.setInt(HConstants.HBASE_CLIENT_RETRIES_NUMBER, 1); TestFilterWrapper.conf.setInt(HConstants.HBASE_CLIENT_RETRIES_NUMBER, 1);
try { try {
admin = new HBaseAdmin(conf); connection = ConnectionFactory.createConnection(TestFilterWrapper.conf);
admin = TEST_UTIL.getHBaseAdmin();
} catch (MasterNotRunningException e) { } catch (MasterNotRunningException e) {
assertNull("Master is not running", e); assertNull("Master is not running", e);
} catch (ZooKeeperConnectionException e) { } catch (ZooKeeperConnectionException e) {
@ -187,7 +191,6 @@ public class TestFilterWrapper {
@BeforeClass @BeforeClass
public static void setUp() throws Exception { public static void setUp() throws Exception {
Configuration config = TEST_UTIL.getConfiguration();
TEST_UTIL.startMiniCluster(1); TEST_UTIL.startMiniCluster(1);
initialize(TEST_UTIL.getConfiguration()); initialize(TEST_UTIL.getConfiguration());
} }
@ -195,6 +198,7 @@ public class TestFilterWrapper {
@AfterClass @AfterClass
public static void tearDown() throws Exception { public static void tearDown() throws Exception {
deleteTable(); deleteTable();
connection.close();
TEST_UTIL.shutdownMiniCluster(); TEST_UTIL.shutdownMiniCluster();
} }

View File

@ -22,6 +22,7 @@ package org.apache.hadoop.hbase.filter;
import org.apache.commons.logging.Log; import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory; import org.apache.commons.logging.LogFactory;
import org.apache.commons.codec.binary.Hex; 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.Put;
import org.apache.hadoop.hbase.client.Result; import org.apache.hadoop.hbase.client.Result;
import org.apache.hadoop.hbase.client.ResultScanner; import org.apache.hadoop.hbase.client.ResultScanner;
@ -48,7 +49,7 @@ public class TestScanRowPrefix extends FilterTestingCluster {
@Test @Test
public void testPrefixScanning() throws IOException { public void testPrefixScanning() throws IOException {
String tableName = "prefixScanning"; TableName tableName = TableName.valueOf("prefixScanning");
createTable(tableName,"F"); createTable(tableName,"F");
Table table = openTable(tableName); Table table = openTable(tableName);

View File

@ -16,15 +16,6 @@
*/ */
package org.apache.hadoop.hbase.io.encoding; 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.Log;
import org.apache.commons.logging.LogFactory; import org.apache.commons.logging.LogFactory;
import org.apache.hadoop.conf.Configuration; 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.Durability;
import org.apache.hadoop.hbase.client.Get; import org.apache.hadoop.hbase.client.Get;
import org.apache.hadoop.hbase.client.HBaseAdmin; 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.Put;
import org.apache.hadoop.hbase.client.Result; import org.apache.hadoop.hbase.client.Result;
import org.apache.hadoop.hbase.client.Table; import org.apache.hadoop.hbase.client.Table;
@ -55,6 +45,15 @@ import org.junit.BeforeClass;
import org.junit.Test; import org.junit.Test;
import org.junit.experimental.categories.Category; 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. * Tests changing data block encoding settings of a column family.
*/ */
@ -127,7 +126,7 @@ public class TestChangingEncoding {
+ "_col" + j); + "_col" + j);
} }
static void writeTestDataBatch(Configuration conf, TableName tableName, static void writeTestDataBatch(TableName tableName,
int batchId) throws Exception { int batchId) throws Exception {
LOG.debug("Writing test data batch " + batchId); LOG.debug("Writing test data batch " + batchId);
List<Put> puts = new ArrayList<>(); List<Put> 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 { int batchId) throws Exception {
LOG.debug("Verifying test data batch " + batchId); 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) { for (int i = 0; i < NUM_ROWS_PER_BATCH; ++i) {
Get get = new Get(getRowKey(batchId, i)); Get get = new Get(getRowKey(batchId, i));
Result result = table.get(get); Result result = table.get(get);
@ -162,13 +161,13 @@ public class TestChangingEncoding {
} }
private void writeSomeNewData() throws Exception { private void writeSomeNewData() throws Exception {
writeTestDataBatch(conf, tableName, numBatchesWritten); writeTestDataBatch(tableName, numBatchesWritten);
++numBatchesWritten; ++numBatchesWritten;
} }
private void verifyAllData() throws Exception { private void verifyAllData() throws Exception {
for (int i = 0; i < numBatchesWritten; ++i) { for (int i = 0; i < numBatchesWritten; ++i) {
verifyTestDataBatch(conf, tableName, i); verifyTestDataBatch(tableName, i);
} }
} }

View File

@ -68,14 +68,14 @@ public class TestLoadAndSwitchEncodeOnDisk extends
@Test(timeout=TIMEOUT_MS) @Test(timeout=TIMEOUT_MS)
public void loadTest() throws Exception { public void loadTest() throws Exception {
HBaseAdmin admin = new HBaseAdmin(conf); HBaseAdmin admin = TEST_UTIL.getHBaseAdmin();
compression = Compression.Algorithm.GZ; // used for table setup compression = Compression.Algorithm.GZ; // used for table setup
super.loadTest(); super.loadTest();
HColumnDescriptor hcd = getColumnDesc(admin); HColumnDescriptor hcd = getColumnDesc(admin);
System.err.println("\nDisabling encode-on-disk. Old column descriptor: " + hcd + "\n"); 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); assertAllOnLine(t);
admin.disableTable(TABLE); admin.disableTable(TABLE);

View File

@ -26,6 +26,7 @@ import java.io.IOException;
import org.apache.commons.logging.Log; import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory; import org.apache.commons.logging.LogFactory;
import org.apache.hadoop.fs.FileUtil; 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.LargeTests;
import org.apache.hadoop.hbase.testclassification.MapReduceTests; import org.apache.hadoop.hbase.testclassification.MapReduceTests;
import org.apache.hadoop.hbase.client.HTable; import org.apache.hadoop.hbase.client.HTable;
@ -73,7 +74,7 @@ public class TestTableMapReduce extends TestTableMapReduceBase {
} }
@Override @Override
protected void runTestOnTable(HTable table) throws IOException { protected void runTestOnTable(Table table) throws IOException {
JobConf jobConf = null; JobConf jobConf = null;
try { try {
LOG.info("Before map/reduce startup"); LOG.info("Before map/reduce startup");

View File

@ -28,6 +28,7 @@ import org.apache.hadoop.hbase.TableName;
import org.apache.hadoop.hbase.client.Admin; import org.apache.hadoop.hbase.client.Admin;
import org.apache.hadoop.hbase.client.HTable; import org.apache.hadoop.hbase.client.HTable;
import org.apache.hadoop.hbase.client.Result; 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.io.ImmutableBytesWritable;
import org.apache.hadoop.hbase.master.snapshot.SnapshotManager; import org.apache.hadoop.hbase.master.snapshot.SnapshotManager;
import org.apache.hadoop.hbase.snapshot.SnapshotTestingUtils; import org.apache.hadoop.hbase.snapshot.SnapshotTestingUtils;
@ -152,7 +153,7 @@ public abstract class TableSnapshotInputFormatTestBase {
Admin admin = util.getHBaseAdmin(); Admin admin = util.getHBaseAdmin();
// put some stuff in the table // put some stuff in the table
HTable table = new HTable(util.getConfiguration(), tableName); Table table = util.getConnection().getTable(tableName);
util.loadTable(table, FAMILIES); util.loadTable(table, FAMILIES);
Path rootDir = FSUtils.getRootDir(util.getConfiguration()); Path rootDir = FSUtils.getRootDir(util.getConfiguration());

View File

@ -24,6 +24,7 @@ import static org.junit.Assert.assertNotNull;
import static org.junit.Assert.assertNotSame; import static org.junit.Assert.assertNotSame;
import static org.junit.Assert.assertTrue; import static org.junit.Assert.assertTrue;
import static org.junit.Assert.fail; import static org.junit.Assert.fail;
import static org.mockito.Mockito.when;
import java.io.IOException; import java.io.IOException;
import java.util.Arrays; 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.PerformanceEvaluation;
import org.apache.hadoop.hbase.TableDescriptor; import org.apache.hadoop.hbase.TableDescriptor;
import org.apache.hadoop.hbase.TableName; 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.HBaseAdmin;
import org.apache.hadoop.hbase.client.HRegionLocator; import org.apache.hadoop.hbase.client.HRegionLocator;
import org.apache.hadoop.hbase.client.HTable; import org.apache.hadoop.hbase.client.HTable;
@ -378,7 +380,7 @@ public class TestHFileOutputFormat {
try { try {
util.startMiniCluster(); util.startMiniCluster();
Path testDir = util.getDataTestDirOnTestFS("testLocalMRIncrementalLoad"); Path testDir = util.getDataTestDirOnTestFS("testLocalMRIncrementalLoad");
admin = new HBaseAdmin(conf); admin = util.getHBaseAdmin();
HTable table = util.createTable(TABLE_NAME, FAMILIES); HTable table = util.createTable(TABLE_NAME, FAMILIES);
assertEquals("Should start with empty table", assertEquals("Should start with empty table",
0, util.countRows(table)); 0, util.countRows(table));
@ -408,7 +410,7 @@ public class TestHFileOutputFormat {
// handle the split case // handle the split case
if (shouldChangeRegions) { if (shouldChangeRegions) {
LOG.info("Changing regions in table"); LOG.info("Changing regions in table");
admin.disableTable(table.getTableName()); admin.disableTable(table.getName());
while(util.getMiniHBaseCluster().getMaster().getAssignmentManager(). while(util.getMiniHBaseCluster().getMaster().getAssignmentManager().
getRegionStates().isRegionsInTransition()) { getRegionStates().isRegionsInTransition()) {
Threads.sleep(200); Threads.sleep(200);
@ -417,9 +419,9 @@ public class TestHFileOutputFormat {
byte[][] newStartKeys = generateRandomStartKeys(15); byte[][] newStartKeys = generateRandomStartKeys(15);
util.createMultiRegions( util.createMultiRegions(
util.getConfiguration(), table, FAMILIES[0], newStartKeys); util.getConfiguration(), table, FAMILIES[0], newStartKeys);
admin.enableTable(table.getTableName()); admin.enableTable(table.getName());
while (table.getRegionLocations().size() != 15 || while (table.getRegionLocations().size() != 15 ||
!admin.isTableAvailable(table.getTableName())) { !admin.isTableAvailable(table.getName())) {
Thread.sleep(200); Thread.sleep(200);
LOG.info("Waiting for new region assignment to happen"); LOG.info("Waiting for new region assignment to happen");
} }
@ -967,7 +969,7 @@ public class TestHFileOutputFormat {
util.startMiniCluster(); util.startMiniCluster();
Path testDir = util.getDataTestDirOnTestFS("testExcludeMinorCompaction"); Path testDir = util.getDataTestDirOnTestFS("testExcludeMinorCompaction");
final FileSystem fs = util.getDFSCluster().getFileSystem(); final FileSystem fs = util.getDFSCluster().getFileSystem();
HBaseAdmin admin = new HBaseAdmin(conf); Admin admin = util.getHBaseAdmin();
HTable table = util.createTable(TABLE_NAME, FAMILIES); HTable table = util.createTable(TABLE_NAME, FAMILIES);
assertEquals("Should start with empty table", 0, util.countRows(table)); assertEquals("Should start with empty table", 0, util.countRows(table));
@ -982,7 +984,7 @@ public class TestHFileOutputFormat {
Put p = new Put(Bytes.toBytes("test")); Put p = new Put(Bytes.toBytes("test"));
p.add(FAMILIES[0], Bytes.toBytes("1"), Bytes.toBytes("1")); p.add(FAMILIES[0], Bytes.toBytes("1"), Bytes.toBytes("1"));
table.put(p); table.put(p);
admin.flush(TABLE_NAME.getName()); admin.flush(TABLE_NAME);
assertEquals(1, util.countRows(table)); assertEquals(1, util.countRows(table));
quickPoll(new Callable<Boolean>() { quickPoll(new Callable<Boolean>() {
public Boolean call() throws Exception { public Boolean call() throws Exception {
@ -1008,7 +1010,7 @@ public class TestHFileOutputFormat {
assertEquals(2, fs.listStatus(storePath).length); assertEquals(2, fs.listStatus(storePath).length);
// minor compactions shouldn't get rid of the file // minor compactions shouldn't get rid of the file
admin.compact(TABLE_NAME.getName()); admin.compact(TABLE_NAME);
try { try {
quickPoll(new Callable<Boolean>() { quickPoll(new Callable<Boolean>() {
public Boolean call() throws Exception { public Boolean call() throws Exception {
@ -1021,7 +1023,7 @@ public class TestHFileOutputFormat {
} }
// a major compaction should work though // a major compaction should work though
admin.majorCompact(TABLE_NAME.getName()); admin.majorCompact(TABLE_NAME);
quickPoll(new Callable<Boolean>() { quickPoll(new Callable<Boolean>() {
public Boolean call() throws Exception { public Boolean call() throws Exception {
return fs.listStatus(storePath).length == 1; return fs.listStatus(storePath).length == 1;
@ -1063,7 +1065,7 @@ public class TestHFileOutputFormat {
admin.enableTable(tname); admin.enableTable(tname);
} else if ("incremental".equals(args[0])) { } else if ("incremental".equals(args[0])) {
TableName tname = TableName.valueOf(args[1]); 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"); Path outDir = new Path("incremental-out");
runIncrementalPELoad(conf, table, outDir); runIncrementalPELoad(conf, table, outDir);
} else { } else {

View File

@ -255,7 +255,7 @@ public class TestImportExport {
.setMaxVersions(1) .setMaxVersions(1)
); );
UTIL.getHBaseAdmin().createTable(desc); UTIL.getHBaseAdmin().createTable(desc);
Table t = new HTable(UTIL.getConfiguration(), desc.getTableName()); Table t = UTIL.getConnection().getTable(desc.getTableName());
Put p = new Put(ROW1); Put p = new Put(ROW1);
p.add(FAMILYA, QUAL, now, QUAL); p.add(FAMILYA, QUAL, now, QUAL);
@ -286,7 +286,7 @@ public class TestImportExport {
.setKeepDeletedCells(true) .setKeepDeletedCells(true)
); );
UTIL.getHBaseAdmin().createTable(desc); UTIL.getHBaseAdmin().createTable(desc);
Table t = new HTable(UTIL.getConfiguration(), desc.getTableName()); Table t = UTIL.getConnection().getTable(desc.getTableName());
Put p = new Put(ROW1); Put p = new Put(ROW1);
p.add(FAMILYA, QUAL, now, QUAL); p.add(FAMILYA, QUAL, now, QUAL);
@ -318,7 +318,7 @@ public class TestImportExport {
); );
UTIL.getHBaseAdmin().createTable(desc); UTIL.getHBaseAdmin().createTable(desc);
t.close(); t.close();
t = new HTable(UTIL.getConfiguration(), desc.getTableName()); t = UTIL.getConnection().getTable(desc.getTableName());
args = new String[] { args = new String[] {
IMPORT_TABLE, IMPORT_TABLE,
FQ_OUTPUT_DIR FQ_OUTPUT_DIR
@ -344,14 +344,16 @@ public class TestImportExport {
@Test @Test
public void testWithMultipleDeleteFamilyMarkersOfSameRowSameFamily() throws Exception { public void testWithMultipleDeleteFamilyMarkersOfSameRowSameFamily() throws Exception {
String EXPORT_TABLE = "exportWithMultipleDeleteFamilyMarkersOfSameRowSameFamily"; TableName EXPORT_TABLE =
HTableDescriptor desc = new HTableDescriptor(TableName.valueOf(EXPORT_TABLE)); TableName.valueOf("exportWithMultipleDeleteFamilyMarkersOfSameRowSameFamily");
HTableDescriptor desc = new HTableDescriptor(EXPORT_TABLE);
desc.addFamily(new HColumnDescriptor(FAMILYA) desc.addFamily(new HColumnDescriptor(FAMILYA)
.setMaxVersions(5) .setMaxVersions(5)
.setKeepDeletedCells(true) .setKeepDeletedCells(true)
); );
UTIL.getHBaseAdmin().createTable(desc); UTIL.getHBaseAdmin().createTable(desc);
HTable exportT = new HTable(UTIL.getConfiguration(), EXPORT_TABLE);
Table exportT = UTIL.getConnection().getTable(EXPORT_TABLE);
//Add first version of QUAL //Add first version of QUAL
Put p = new Put(ROW1); Put p = new Put(ROW1);
@ -373,8 +375,7 @@ public class TestImportExport {
String[] args = new String[] { String[] args = new String[] {
"-D" + Export.RAW_SCAN + "=true", "-D" + Export.RAW_SCAN + "=true", EXPORT_TABLE.getNameAsString(),
EXPORT_TABLE,
FQ_OUTPUT_DIR, FQ_OUTPUT_DIR,
"1000", // max number of key versions per key to export "1000", // max number of key versions per key to export
}; };
@ -388,7 +389,7 @@ public class TestImportExport {
); );
UTIL.getHBaseAdmin().createTable(desc); UTIL.getHBaseAdmin().createTable(desc);
HTable importT = new HTable(UTIL.getConfiguration(), IMPORT_TABLE); Table importT = UTIL.getConnection().getTable(TableName.valueOf(IMPORT_TABLE));
args = new String[] { args = new String[] {
IMPORT_TABLE, IMPORT_TABLE,
FQ_OUTPUT_DIR FQ_OUTPUT_DIR
@ -429,7 +430,7 @@ public class TestImportExport {
HTableDescriptor desc = new HTableDescriptor(TableName.valueOf(EXPORT_TABLE)); HTableDescriptor desc = new HTableDescriptor(TableName.valueOf(EXPORT_TABLE));
desc.addFamily(new HColumnDescriptor(FAMILYA).setMaxVersions(5)); desc.addFamily(new HColumnDescriptor(FAMILYA).setMaxVersions(5));
UTIL.getHBaseAdmin().createTable(desc); UTIL.getHBaseAdmin().createTable(desc);
Table exportTable = new HTable(UTIL.getConfiguration(), desc.getTableName()); Table exportTable = UTIL.getConnection().getTable(desc.getTableName());
Put p1 = new Put(ROW1); Put p1 = new Put(ROW1);
p1.add(FAMILYA, QUAL, now, QUAL); p1.add(FAMILYA, QUAL, now, QUAL);
@ -454,7 +455,7 @@ public class TestImportExport {
desc.addFamily(new HColumnDescriptor(FAMILYA).setMaxVersions(5)); desc.addFamily(new HColumnDescriptor(FAMILYA).setMaxVersions(5));
UTIL.getHBaseAdmin().createTable(desc); 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(), 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, "-D" + Import.FILTER_ARGS_CONF_KEY + "=" + Bytes.toString(ROW1), IMPORT_TABLE, FQ_OUTPUT_DIR,
"1000" }; "1000" };

View File

@ -39,6 +39,8 @@ import org.apache.hadoop.hbase.HBaseTestingUtility;
import org.apache.hadoop.hbase.HConstants; import org.apache.hadoop.hbase.HConstants;
import org.apache.hadoop.hbase.TableName; import org.apache.hadoop.hbase.TableName;
import org.apache.hadoop.hbase.client.Admin; 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.LargeTests;
import org.apache.hadoop.hbase.testclassification.MapReduceTests; import org.apache.hadoop.hbase.testclassification.MapReduceTests;
import org.apache.hadoop.hbase.client.Durability; 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.master.classes", OperationAttributesTestController.class.getName());
conf.set("hbase.coprocessor.region.classes", OperationAttributesTestController.class.getName()); conf.set("hbase.coprocessor.region.classes", OperationAttributesTestController.class.getName());
util.startMiniCluster(); util.startMiniCluster();
Admin admin = new HBaseAdmin(util.getConfiguration());
util.startMiniMapReduceCluster(); util.startMiniMapReduceCluster();
} }
@ -196,7 +197,8 @@ public class TestImportTSVWithOperationAttributes implements Configurable {
int valueMultiplier, boolean dataAvailable) throws IOException { int valueMultiplier, boolean dataAvailable) throws IOException {
LOG.debug("Validating table."); LOG.debug("Validating table.");
Table table = new HTable(conf, tableName); Connection connection = ConnectionFactory.createConnection(conf);
Table table = connection.getTable(tableName);
boolean verified = false; boolean verified = false;
long pause = conf.getLong("hbase.client.pause", 5 * 1000); long pause = conf.getLong("hbase.client.pause", 5 * 1000);
int numRetries = conf.getInt(HConstants.HBASE_CLIENT_RETRIES_NUMBER, 5); int numRetries = conf.getInt(HConstants.HBASE_CLIENT_RETRIES_NUMBER, 5);
@ -238,6 +240,7 @@ public class TestImportTSVWithOperationAttributes implements Configurable {
} }
} }
table.close(); table.close();
connection.close();
assertTrue(verified); assertTrue(verified);
} }

View File

@ -122,7 +122,6 @@ public class TestImportTSVWithVisibilityLabels implements Configurable {
// Wait for the labels table to become available // Wait for the labels table to become available
util.waitTableEnabled(VisibilityConstants.LABELS_TABLE_NAME.getName(), 50000); util.waitTableEnabled(VisibilityConstants.LABELS_TABLE_NAME.getName(), 50000);
createLabels(); createLabels();
Admin admin = new HBaseAdmin(util.getConfiguration());
util.startMiniMapReduceCluster(); util.startMiniMapReduceCluster();
} }
@ -185,7 +184,7 @@ public class TestImportTSVWithVisibilityLabels implements Configurable {
private void issueDeleteAndVerifyData(TableName tableName) throws IOException { private void issueDeleteAndVerifyData(TableName tableName) throws IOException {
LOG.debug("Validating table after delete."); LOG.debug("Validating table after delete.");
Table table = new HTable(conf, tableName); Table table = util.getConnection().getTable(tableName);
boolean verified = false; boolean verified = false;
long pause = conf.getLong("hbase.client.pause", 5 * 1000); long pause = conf.getLong("hbase.client.pause", 5 * 1000);
int numRetries = conf.getInt(HConstants.HBASE_CLIENT_RETRIES_NUMBER, 5); int numRetries = conf.getInt(HConstants.HBASE_CLIENT_RETRIES_NUMBER, 5);
@ -370,7 +369,7 @@ public class TestImportTSVWithVisibilityLabels implements Configurable {
int valueMultiplier) throws IOException { int valueMultiplier) throws IOException {
LOG.debug("Validating table."); LOG.debug("Validating table.");
Table table = new HTable(conf, tableName); Table table = util.getConnection().getTable(tableName);
boolean verified = false; boolean verified = false;
long pause = conf.getLong("hbase.client.pause", 5 * 1000); long pause = conf.getLong("hbase.client.pause", 5 * 1000);
int numRetries = conf.getInt(HConstants.HBASE_CLIENT_RETRIES_NUMBER, 5); int numRetries = conf.getInt(HConstants.HBASE_CLIENT_RETRIES_NUMBER, 5);

View File

@ -43,6 +43,8 @@ import org.apache.hadoop.hbase.HBaseTestingUtility;
import org.apache.hadoop.hbase.HConstants; import org.apache.hadoop.hbase.HConstants;
import org.apache.hadoop.hbase.TableName; import org.apache.hadoop.hbase.TableName;
import org.apache.hadoop.hbase.TableNotFoundException; 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.HTable;
import org.apache.hadoop.hbase.client.Result; import org.apache.hadoop.hbase.client.Result;
import org.apache.hadoop.hbase.client.ResultScanner; import org.apache.hadoop.hbase.client.ResultScanner;
@ -346,7 +348,8 @@ public class TestImportTsv implements Configurable {
String family, int valueMultiplier) throws IOException { String family, int valueMultiplier) throws IOException {
LOG.debug("Validating table."); LOG.debug("Validating table.");
Table table = new HTable(conf, tableName); Connection connection = ConnectionFactory.createConnection(conf);
Table table = connection.getTable(tableName);
boolean verified = false; boolean verified = false;
long pause = conf.getLong("hbase.client.pause", 5 * 1000); long pause = conf.getLong("hbase.client.pause", 5 * 1000);
int numRetries = conf.getInt(HConstants.HBASE_CLIENT_RETRIES_NUMBER, 5); int numRetries = conf.getInt(HConstants.HBASE_CLIENT_RETRIES_NUMBER, 5);
@ -378,6 +381,7 @@ public class TestImportTsv implements Configurable {
} }
} }
table.close(); table.close();
connection.close();
assertTrue(verified); assertTrue(verified);
} }

View File

@ -255,7 +255,7 @@ public class TestLoadIncrementalHFiles {
String [] args= {dir.toString(), tableName.toString()}; String [] args= {dir.toString(), tableName.toString()};
loader.run(args); loader.run(args);
Table table = new HTable(util.getConfiguration(), tableName); Table table = util.getConnection().getTable(tableName);
try { try {
assertEquals(expectedRows, util.countRows(table)); assertEquals(expectedRows, util.countRows(table));
} finally { } finally {

View File

@ -242,7 +242,7 @@ public class TestLoadIncrementalHFilesSplitRecovery {
assertEquals(htds.length, 1); assertEquals(htds.length, 1);
Table t = null; Table t = null;
try { try {
t = new HTable(util.getConfiguration(), table); t = util.getConnection().getTable(table);
Scan s = new Scan(); Scan s = new Scan();
ResultScanner sr = t.getScanner(s); ResultScanner sr = t.getScanner(s);
int i = 0; int i = 0;

View File

@ -126,11 +126,10 @@ public class TestMultithreadedTableMapper {
@Test @Test
public void testMultithreadedTableMapper() public void testMultithreadedTableMapper()
throws IOException, InterruptedException, ClassNotFoundException { throws IOException, InterruptedException, ClassNotFoundException {
runTestOnTable(new HTable(new Configuration(UTIL.getConfiguration()), runTestOnTable(UTIL.getConnection().getTable(MULTI_REGION_TABLE_NAME));
MULTI_REGION_TABLE_NAME));
} }
private void runTestOnTable(HTable table) private void runTestOnTable(Table table)
throws IOException, InterruptedException, ClassNotFoundException { throws IOException, InterruptedException, ClassNotFoundException {
Job job = null; Job job = null;
try { try {
@ -140,16 +139,16 @@ public class TestMultithreadedTableMapper {
Scan scan = new Scan(); Scan scan = new Scan();
scan.addFamily(INPUT_FAMILY); scan.addFamily(INPUT_FAMILY);
TableMapReduceUtil.initTableMapperJob( TableMapReduceUtil.initTableMapperJob(
table.getTableName(), scan, table.getName(), scan,
MultithreadedTableMapper.class, ImmutableBytesWritable.class, MultithreadedTableMapper.class, ImmutableBytesWritable.class,
Put.class, job); Put.class, job);
MultithreadedTableMapper.setMapperClass(job, ProcessContentsMapper.class); MultithreadedTableMapper.setMapperClass(job, ProcessContentsMapper.class);
MultithreadedTableMapper.setNumberOfThreads(job, NUMBER_OF_THREADS); MultithreadedTableMapper.setNumberOfThreads(job, NUMBER_OF_THREADS);
TableMapReduceUtil.initTableReducerJob( TableMapReduceUtil.initTableReducerJob(
Bytes.toString(table.getTableName()), table.getName().getNameAsString(),
IdentityTableReducer.class, job); IdentityTableReducer.class, job);
FileOutputFormat.setOutputPath(job, new Path("test")); FileOutputFormat.setOutputPath(job, new Path("test"));
LOG.info("Started " + table.getTableName()); LOG.info("Started " + table.getName());
assertTrue(job.waitForCompletion(true)); assertTrue(job.waitForCompletion(true));
LOG.info("After map/reduce completion"); LOG.info("After map/reduce completion");
// verify map-reduce results // verify map-reduce results
@ -164,7 +163,7 @@ public class TestMultithreadedTableMapper {
} }
private void verify(TableName tableName) throws IOException { 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; boolean verified = false;
long pause = UTIL.getConfiguration().getLong("hbase.client.pause", 5 * 1000); long pause = UTIL.getConfiguration().getLong("hbase.client.pause", 5 * 1000);
int numRetries = UTIL.getConfiguration().getInt(HConstants.HBASE_CLIENT_RETRIES_NUMBER, 5); int numRetries = UTIL.getConfiguration().getInt(HConstants.HBASE_CLIENT_RETRIES_NUMBER, 5);

View File

@ -33,6 +33,7 @@ import org.apache.hadoop.hbase.client.HTable;
import org.apache.hadoop.hbase.client.Put; import org.apache.hadoop.hbase.client.Put;
import org.apache.hadoop.hbase.client.Result; import org.apache.hadoop.hbase.client.Result;
import org.apache.hadoop.hbase.client.Scan; 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.io.ImmutableBytesWritable;
import org.apache.hadoop.hbase.testclassification.LargeTests; import org.apache.hadoop.hbase.testclassification.LargeTests;
import org.apache.hadoop.hbase.testclassification.VerySlowMapReduceTests; 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; Job job = null;
try { try {
LOG.info("Before map/reduce startup"); LOG.info("Before map/reduce startup");
@ -99,14 +100,14 @@ public class TestTableMapReduce extends TestTableMapReduceBase {
Scan scan = new Scan(); Scan scan = new Scan();
scan.addFamily(INPUT_FAMILY); scan.addFamily(INPUT_FAMILY);
TableMapReduceUtil.initTableMapperJob( TableMapReduceUtil.initTableMapperJob(
Bytes.toString(table.getTableName()), scan, table.getName().getNameAsString(), scan,
ProcessContentsMapper.class, ImmutableBytesWritable.class, ProcessContentsMapper.class, ImmutableBytesWritable.class,
Put.class, job); Put.class, job);
TableMapReduceUtil.initTableReducerJob( TableMapReduceUtil.initTableReducerJob(
Bytes.toString(table.getTableName()), table.getName().getNameAsString(),
IdentityTableReducer.class, job); IdentityTableReducer.class, job);
FileOutputFormat.setOutputPath(job, new Path("test")); FileOutputFormat.setOutputPath(job, new Path("test"));
LOG.info("Started " + Bytes.toString(table.getTableName())); LOG.info("Started " + table.getName().getNameAsString());
assertTrue(job.waitForCompletion(true)); assertTrue(job.waitForCompletion(true));
LOG.info("After map/reduce completion"); LOG.info("After map/reduce completion");

View File

@ -71,7 +71,7 @@ public abstract class TestTableMapReduceBase {
/** /**
* Handles API-specifics for setting up and executing the job. * 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 @BeforeClass
public static void beforeClass() throws Exception { public static void beforeClass() throws Exception {
@ -95,7 +95,7 @@ public abstract class TestTableMapReduceBase {
*/ */
@Test @Test
public void testMultiRegionTable() throws IOException { public void testMultiRegionTable() throws IOException {
runTestOnTable(new HTable(UTIL.getConfiguration(), MULTI_REGION_TABLE_NAME)); runTestOnTable(UTIL.getConnection().getTable(MULTI_REGION_TABLE_NAME));
} }
@Test @Test
@ -103,7 +103,7 @@ public abstract class TestTableMapReduceBase {
Configuration conf = new Configuration(UTIL.getConfiguration()); Configuration conf = new Configuration(UTIL.getConfiguration());
// force use of combiner for testing purposes // force use of combiner for testing purposes
conf.setInt("mapreduce.map.combine.minspills", 1); 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 { protected void verify(TableName tableName) throws IOException {
Table table = new HTable(UTIL.getConfiguration(), tableName); Table table = UTIL.getConnection().getTable(tableName);
boolean verified = false; boolean verified = false;
long pause = UTIL.getConfiguration().getLong("hbase.client.pause", 5 * 1000); long pause = UTIL.getConfiguration().getLong("hbase.client.pause", 5 * 1000);
int numRetries = UTIL.getConfiguration().getInt(HConstants.HBASE_CLIENT_RETRIES_NUMBER, 5); int numRetries = UTIL.getConfiguration().getInt(HConstants.HBASE_CLIENT_RETRIES_NUMBER, 5);

View File

@ -18,28 +18,25 @@
*/ */
package org.apache.hadoop.hbase.mapreduce; 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.Log;
import org.apache.commons.logging.LogFactory; import org.apache.commons.logging.LogFactory;
import org.apache.hadoop.conf.Configurable; import org.apache.hadoop.conf.Configurable;
import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.fs.FileUtil; 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.Admin;
import org.apache.hadoop.hbase.client.HBaseAdmin; import org.apache.hadoop.hbase.client.Connection;
import org.apache.hadoop.hbase.client.HTable; 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.Put;
import org.apache.hadoop.hbase.client.Result; import org.apache.hadoop.hbase.client.Result;
import org.apache.hadoop.hbase.client.ResultScanner; import org.apache.hadoop.hbase.client.ResultScanner;
import org.apache.hadoop.hbase.client.Scan; 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.client.Table;
import org.apache.hadoop.hbase.io.ImmutableBytesWritable; import org.apache.hadoop.hbase.io.ImmutableBytesWritable;
import org.apache.hadoop.hbase.testclassification.LargeTests; 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.io.Text;
import org.apache.hadoop.mapreduce.Job; import org.apache.hadoop.mapreduce.Job;
import org.apache.hadoop.mapreduce.lib.output.NullOutputFormat; import org.apache.hadoop.mapreduce.lib.output.NullOutputFormat;
import org.junit.After;
import org.junit.AfterClass; import org.junit.AfterClass;
import org.junit.Before; import org.junit.Before;
import org.junit.BeforeClass; import org.junit.BeforeClass;
import org.junit.Test; import org.junit.Test;
import org.junit.experimental.categories.Category; 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}) @Category({MapReduceTests.class, LargeTests.class})
public class TestTimeRangeMapRed { public class TestTimeRangeMapRed {
private final static Log log = LogFactory.getLog(TestTimeRangeMapRed.class); private final static Log log = LogFactory.getLog(TestTimeRangeMapRed.class);
@ -94,12 +98,7 @@ public class TestTimeRangeMapRed {
@Before @Before
public void before() throws Exception { public void before() throws Exception {
this.admin = new HBaseAdmin(UTIL.getConfiguration()); this.admin = UTIL.getHBaseAdmin();
}
@After
public void after() throws IOException {
this.admin.close();
} }
private static class ProcessTimeRangeMapper private static class ProcessTimeRangeMapper
@ -137,7 +136,8 @@ public class TestTimeRangeMapRed {
public void setConf(Configuration configuration) { public void setConf(Configuration configuration) {
this.conf = configuration; this.conf = configuration;
try { try {
table = new HTable(HBaseConfiguration.create(conf), TABLE_NAME); Connection connection = ConnectionFactory.createConnection(conf);
table = connection.getTable(TABLE_NAME);
} catch (IOException e) { } catch (IOException e) {
e.printStackTrace(); e.printStackTrace();
} }
@ -159,7 +159,7 @@ public class TestTimeRangeMapRed {
put.add(FAMILY_NAME, COLUMN_NAME, entry.getKey(), Bytes.toBytes(false)); put.add(FAMILY_NAME, COLUMN_NAME, entry.getKey(), Bytes.toBytes(false));
puts.add(put); puts.add(put);
} }
Table table = new HTable(UTIL.getConfiguration(), desc.getTableName()); Table table = UTIL.getConnection().getTable(desc.getTableName());
table.put(puts); table.put(puts);
runTestOnTable(); runTestOnTable();
verify(table); verify(table);
@ -204,7 +204,7 @@ public class TestTimeRangeMapRed {
+ "\t" + Bytes.toString(CellUtil.cloneQualifier(kv)) + "\t" + Bytes.toString(CellUtil.cloneQualifier(kv))
+ "\t" + kv.getTimestamp() + "\t" + Bytes.toBoolean(CellUtil.cloneValue(kv))); + "\t" + kv.getTimestamp() + "\t" + Bytes.toBoolean(CellUtil.cloneValue(kv)));
org.junit.Assert.assertEquals(TIMESTAMP.get(kv.getTimestamp()), org.junit.Assert.assertEquals(TIMESTAMP.get(kv.getTimestamp()),
(Boolean)Bytes.toBoolean(CellUtil.cloneValue(kv))); Bytes.toBoolean(CellUtil.cloneValue(kv)));
} }
} }
scanner.close(); scanner.close();

View File

@ -205,7 +205,7 @@ public class TestAssignmentListener {
assertEquals(0, listener.getCloseCount()); assertEquals(0, listener.getCloseCount());
// Add some data // Add some data
Table table = new HTable(TEST_UTIL.getConfiguration(), TABLE_NAME); Table table = TEST_UTIL.getConnection().getTable(TABLE_NAME);
try { try {
for (int i = 0; i < 10; ++i) { for (int i = 0; i < 10; ++i) {
byte[] key = Bytes.toBytes("row-" + i); byte[] key = Bytes.toBytes("row-" + i);

View File

@ -181,7 +181,7 @@ public class TestAssignmentManagerOnCluster {
desc.addFamily(new HColumnDescriptor(FAMILY)); desc.addFamily(new HColumnDescriptor(FAMILY));
admin.createTable(desc); 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( HRegionInfo hri = new HRegionInfo(
desc.getTableName(), Bytes.toBytes("A"), Bytes.toBytes("Z")); desc.getTableName(), Bytes.toBytes("A"), Bytes.toBytes("Z"));
MetaTableAccessor.addRegionToMeta(meta, hri); MetaTableAccessor.addRegionToMeta(meta, hri);
@ -224,7 +224,7 @@ public class TestAssignmentManagerOnCluster {
desc.addFamily(new HColumnDescriptor(FAMILY)); desc.addFamily(new HColumnDescriptor(FAMILY));
admin.createTable(desc); 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( final HRegionInfo hri = new HRegionInfo(
desc.getTableName(), Bytes.toBytes("A"), Bytes.toBytes("Z")); desc.getTableName(), Bytes.toBytes("A"), Bytes.toBytes("Z"));
MetaTableAccessor.addRegionToMeta(meta, hri); MetaTableAccessor.addRegionToMeta(meta, hri);
@ -436,7 +436,7 @@ public class TestAssignmentManagerOnCluster {
desc.addFamily(new HColumnDescriptor(FAMILY)); desc.addFamily(new HColumnDescriptor(FAMILY));
admin.createTable(desc); 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( HRegionInfo hri = new HRegionInfo(
desc.getTableName(), Bytes.toBytes("A"), Bytes.toBytes("Z")); desc.getTableName(), Bytes.toBytes("A"), Bytes.toBytes("Z"));
MetaTableAccessor.addRegionToMeta(meta, hri); MetaTableAccessor.addRegionToMeta(meta, hri);
@ -485,7 +485,7 @@ public class TestAssignmentManagerOnCluster {
desc.addFamily(new HColumnDescriptor(FAMILY)); desc.addFamily(new HColumnDescriptor(FAMILY));
admin.createTable(desc); 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( HRegionInfo hri = new HRegionInfo(
desc.getTableName(), Bytes.toBytes("A"), Bytes.toBytes("Z")); desc.getTableName(), Bytes.toBytes("A"), Bytes.toBytes("Z"));
MetaTableAccessor.addRegionToMeta(meta, hri); MetaTableAccessor.addRegionToMeta(meta, hri);
@ -531,7 +531,7 @@ public class TestAssignmentManagerOnCluster {
desc.addFamily(new HColumnDescriptor(FAMILY)); desc.addFamily(new HColumnDescriptor(FAMILY));
admin.createTable(desc); 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( HRegionInfo hri = new HRegionInfo(
desc.getTableName(), Bytes.toBytes("A"), Bytes.toBytes("Z")); desc.getTableName(), Bytes.toBytes("A"), Bytes.toBytes("Z"));
MetaTableAccessor.addRegionToMeta(meta, hri); MetaTableAccessor.addRegionToMeta(meta, hri);
@ -573,7 +573,7 @@ public class TestAssignmentManagerOnCluster {
desc.addFamily(new HColumnDescriptor(FAMILY)); desc.addFamily(new HColumnDescriptor(FAMILY));
admin.createTable(desc); 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( HRegionInfo hri = new HRegionInfo(
desc.getTableName(), Bytes.toBytes("A"), Bytes.toBytes("Z")); desc.getTableName(), Bytes.toBytes("A"), Bytes.toBytes("Z"));
MetaTableAccessor.addRegionToMeta(meta, hri); MetaTableAccessor.addRegionToMeta(meta, hri);
@ -658,7 +658,7 @@ public class TestAssignmentManagerOnCluster {
desc.addFamily(new HColumnDescriptor(FAMILY)); desc.addFamily(new HColumnDescriptor(FAMILY));
admin.createTable(desc); 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( HRegionInfo hri = new HRegionInfo(
desc.getTableName(), Bytes.toBytes("A"), Bytes.toBytes("Z")); desc.getTableName(), Bytes.toBytes("A"), Bytes.toBytes("Z"));
MetaTableAccessor.addRegionToMeta(meta, hri); MetaTableAccessor.addRegionToMeta(meta, hri);
@ -703,7 +703,7 @@ public class TestAssignmentManagerOnCluster {
desc.addFamily(new HColumnDescriptor(FAMILY)); desc.addFamily(new HColumnDescriptor(FAMILY));
admin.createTable(desc); 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( HRegionInfo hri = new HRegionInfo(
desc.getTableName(), Bytes.toBytes("A"), Bytes.toBytes("Z")); desc.getTableName(), Bytes.toBytes("A"), Bytes.toBytes("Z"));
MetaTableAccessor.addRegionToMeta(meta, hri); MetaTableAccessor.addRegionToMeta(meta, hri);
@ -776,7 +776,7 @@ public class TestAssignmentManagerOnCluster {
desc.addFamily(new HColumnDescriptor(FAMILY)); desc.addFamily(new HColumnDescriptor(FAMILY));
admin.createTable(desc); 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( HRegionInfo hri = new HRegionInfo(
desc.getTableName(), Bytes.toBytes("A"), Bytes.toBytes("Z")); desc.getTableName(), Bytes.toBytes("A"), Bytes.toBytes("Z"));
MetaTableAccessor.addRegionToMeta(meta, hri); MetaTableAccessor.addRegionToMeta(meta, hri);
@ -904,7 +904,7 @@ public class TestAssignmentManagerOnCluster {
desc.addFamily(new HColumnDescriptor(FAMILY)); desc.addFamily(new HColumnDescriptor(FAMILY));
admin.createTable(desc); 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( HRegionInfo hri = new HRegionInfo(
desc.getTableName(), Bytes.toBytes("A"), Bytes.toBytes("Z")); desc.getTableName(), Bytes.toBytes("A"), Bytes.toBytes("Z"));
MetaTableAccessor.addRegionToMeta(meta, hri); MetaTableAccessor.addRegionToMeta(meta, hri);
@ -945,7 +945,7 @@ public class TestAssignmentManagerOnCluster {
desc.addFamily(new HColumnDescriptor(FAMILY)); desc.addFamily(new HColumnDescriptor(FAMILY));
admin.createTable(desc); 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( HRegionInfo hri = new HRegionInfo(
desc.getTableName(), Bytes.toBytes("A"), Bytes.toBytes("Z")); desc.getTableName(), Bytes.toBytes("A"), Bytes.toBytes("Z"));
MetaTableAccessor.addRegionToMeta(meta, hri); MetaTableAccessor.addRegionToMeta(meta, hri);
@ -1024,7 +1024,7 @@ public class TestAssignmentManagerOnCluster {
desc.addFamily(new HColumnDescriptor(FAMILY)); desc.addFamily(new HColumnDescriptor(FAMILY));
admin.createTable(desc); 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( HRegionInfo hri = new HRegionInfo(
desc.getTableName(), Bytes.toBytes("A"), Bytes.toBytes("Z")); desc.getTableName(), Bytes.toBytes("A"), Bytes.toBytes("Z"));
MetaTableAccessor.addRegionToMeta(meta, hri); MetaTableAccessor.addRegionToMeta(meta, hri);
@ -1099,7 +1099,7 @@ public class TestAssignmentManagerOnCluster {
HTableDescriptor desc = new HTableDescriptor(TableName.valueOf(table)); HTableDescriptor desc = new HTableDescriptor(TableName.valueOf(table));
desc.addFamily(new HColumnDescriptor(FAMILY)); desc.addFamily(new HColumnDescriptor(FAMILY));
admin.createTable(desc); admin.createTable(desc);
Table meta = new HTable(conf, TableName.META_TABLE_NAME); Table meta = TEST_UTIL.getConnection().getTable(TableName.META_TABLE_NAME);
HRegionInfo hri = HRegionInfo hri =
new HRegionInfo(desc.getTableName(), Bytes.toBytes("A"), Bytes.toBytes("Z")); new HRegionInfo(desc.getTableName(), Bytes.toBytes("A"), Bytes.toBytes("Z"));
MetaTableAccessor.addRegionToMeta(meta, hri); MetaTableAccessor.addRegionToMeta(meta, hri);

View File

@ -253,7 +253,7 @@ public class TestMasterOperationsForRegionReplicas {
ADMIN.disableTable(table); ADMIN.disableTable(table);
// now delete one replica info from all the rows // now delete one replica info from all the rows
// this is to make the meta appear to be only partially updated // 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) { for (byte[] row : tableRows) {
Delete deleteOneReplicaLocation = new Delete(row); Delete deleteOneReplicaLocation = new Delete(row);
deleteOneReplicaLocation.deleteColumns(HConstants.CATALOG_FAMILY, deleteOneReplicaLocation.deleteColumns(HConstants.CATALOG_FAMILY,

View File

@ -99,7 +99,7 @@ public class TestMasterRestartAfterDisablingTable {
TableState.State.DISABLING)); TableState.State.DISABLING));
log("Enabling table\n"); log("Enabling table\n");
// Need a new Admin, the previous one is on the old master // 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.enableTable(table);
admin.close(); admin.close();
log("Waiting for no more RIT\n"); log("Waiting for no more RIT\n");

View File

@ -65,7 +65,7 @@ public class TestMasterTransitions {
TEST_UTIL.startMiniCluster(2); TEST_UTIL.startMiniCluster(2);
// Create a table of three families. This will assign a region. // Create a table of three families. This will assign a region.
TEST_UTIL.createTable(TABLENAME, FAMILIES); 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()); int countOfRegions = TEST_UTIL.createMultiRegions(t, getTestFamily());
TEST_UTIL.waitUntilAllRegionsAssigned(TABLENAME); TEST_UTIL.waitUntilAllRegionsAssigned(TABLENAME);
addToEachStartKey(countOfRegions); addToEachStartKey(countOfRegions);
@ -480,9 +480,8 @@ public class TestMasterTransitions {
* @throws IOException * @throws IOException
*/ */
private static int addToEachStartKey(final int expected) throws IOException { private static int addToEachStartKey(final int expected) throws IOException {
Table t = new HTable(TEST_UTIL.getConfiguration(), TABLENAME); Table t = TEST_UTIL.getConnection().getTable(TABLENAME);
Table meta = new HTable(TEST_UTIL.getConfiguration(), Table meta = TEST_UTIL.getConnection().getTable(TableName.META_TABLE_NAME);
TableName.META_TABLE_NAME);
int rows = 0; int rows = 0;
Scan scan = new Scan(); Scan scan = new Scan();
scan.addColumn(HConstants.CATALOG_FAMILY, HConstants.REGIONINFO_QUALIFIER); scan.addColumn(HConstants.CATALOG_FAMILY, HConstants.REGIONINFO_QUALIFIER);

View File

@ -35,8 +35,6 @@ import org.apache.hadoop.hbase.MiniHBaseCluster;
import org.apache.hadoop.hbase.ServerName; import org.apache.hadoop.hbase.ServerName;
import org.apache.hadoop.hbase.TableExistsException; import org.apache.hadoop.hbase.TableExistsException;
import org.apache.hadoop.hbase.TableName; 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.client.MetaScanner;
import org.apache.hadoop.hbase.testclassification.LargeTests; import org.apache.hadoop.hbase.testclassification.LargeTests;
import org.apache.hadoop.hbase.testclassification.MasterTests; import org.apache.hadoop.hbase.testclassification.MasterTests;
@ -66,8 +64,6 @@ public class TestRestartCluster {
@Test (timeout=300000) @Test (timeout=300000)
public void testClusterRestart() throws Exception { public void testClusterRestart() throws Exception {
UTIL.startMiniCluster(3); UTIL.startMiniCluster(3);
Connection connection = UTIL.getConnection();
while (!UTIL.getMiniHBaseCluster().getMaster().isInitialized()) { while (!UTIL.getMiniHBaseCluster().getMaster().isInitialized()) {
Threads.sleep(1); Threads.sleep(1);
} }
@ -80,7 +76,7 @@ public class TestRestartCluster {
} }
List<HRegionInfo> allRegions = List<HRegionInfo> allRegions =
MetaScanner.listAllRegions(UTIL.getConfiguration(), connection, true); MetaScanner.listAllRegions(UTIL.getConfiguration(), UTIL.getConnection(), true);
assertEquals(4, allRegions.size()); assertEquals(4, allRegions.size());
LOG.info("\n\nShutting down cluster"); 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. // Need to use a new 'Configuration' so we make a new HConnection.
// Otherwise we're reusing an HConnection that has gone stale because // Otherwise we're reusing an HConnection that has gone stale because
// the shutdown of the cluster also called shut of the connection. // the shutdown of the cluster also called shut of the connection.
allRegions = allRegions = MetaScanner
MetaScanner.listAllRegions(new Configuration(UTIL.getConfiguration()), connection, true); .listAllRegions(new Configuration(UTIL.getConfiguration()), UTIL.getConnection(), true);
assertEquals(4, allRegions.size()); assertEquals(4, allRegions.size());
LOG.info("\n\nWaiting for tables to be available"); LOG.info("\n\nWaiting for tables to be available");
for(TableName TABLE: TABLES) { for(TableName TABLE: TABLES) {

View File

@ -290,10 +290,10 @@ public class TestSnapshotFromMaster {
htd.setCompactionEnabled(false); htd.setCompactionEnabled(false);
UTIL.createTable(htd, new byte[][] { TEST_FAM }, UTIL.getConfiguration()); UTIL.createTable(htd, new byte[][] { TEST_FAM }, UTIL.getConfiguration());
// load the table (creates 4 hfiles) // 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); UTIL.flush(TABLE_NAME);
// Put some more data into the table so for sure we get more storefiles. // 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 // disable the table so we can take a snapshot
admin.disableTable(TABLE_NAME); admin.disableTable(TABLE_NAME);

View File

@ -36,6 +36,7 @@ import org.apache.hadoop.hbase.HColumnDescriptor;
import org.apache.hadoop.hbase.HTableDescriptor; import org.apache.hadoop.hbase.HTableDescriptor;
import org.apache.hadoop.hbase.client.Admin; import org.apache.hadoop.hbase.client.Admin;
import org.apache.hadoop.hbase.client.HTable; 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.LargeTests;
import org.apache.hadoop.hbase.testclassification.MasterTests; import org.apache.hadoop.hbase.testclassification.MasterTests;
import org.apache.hadoop.hbase.util.Bytes; 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. // Create a table of three families. This will assign a region.
TEST_UTIL.createTable(TABLENAME, FAMILIES); 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() while(TEST_UTIL.getMiniHBaseCluster().getMaster().getAssignmentManager()
.getRegionStates().getRegionsInTransition().size() > 0) { .getRegionStates().getRegionsInTransition().size() > 0) {
Thread.sleep(100); Thread.sleep(100);

View File

@ -89,7 +89,7 @@ public class TestCompactionState {
Table ht = null; Table ht = null;
try { try {
ht = TEST_UTIL.createTable(table, family); ht = TEST_UTIL.createTable(table, family);
HBaseAdmin admin = new HBaseAdmin(TEST_UTIL.getConfiguration()); HBaseAdmin admin = TEST_UTIL.getHBaseAdmin();
try { try {
admin.compact(table, fakecf); admin.compact(table, fakecf);
} catch (IOException ioe) { } catch (IOException ioe) {
@ -137,7 +137,7 @@ public class TestCompactionState {
int countBefore = countStoreFilesInFamilies(regions, families); int countBefore = countStoreFilesInFamilies(regions, families);
int countBeforeSingleFamily = countStoreFilesInFamily(regions, family); int countBeforeSingleFamily = countStoreFilesInFamily(regions, family);
assertTrue(countBefore > 0); // there should be some data files 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 (expectedState == CompactionState.MINOR) {
if (singleFamily) { if (singleFamily) {
admin.compact(table.getName(), family); admin.compact(table.getName(), family);
@ -154,10 +154,10 @@ public class TestCompactionState {
long curt = System.currentTimeMillis(); long curt = System.currentTimeMillis();
long waitTime = 5000; long waitTime = 5000;
long endt = curt + waitTime; long endt = curt + waitTime;
CompactionState state = admin.getCompactionState(table.getName()); CompactionState state = admin.getCompactionState(table);
while (state == CompactionState.NONE && curt < endt) { while (state == CompactionState.NONE && curt < endt) {
Thread.sleep(10); Thread.sleep(10);
state = admin.getCompactionState(table.getName()); state = admin.getCompactionState(table);
curt = System.currentTimeMillis(); curt = System.currentTimeMillis();
} }
// Now, should have the right compaction state, // Now, should have the right compaction state,
@ -169,10 +169,10 @@ public class TestCompactionState {
} }
} else { } else {
// Wait until the compaction is done // Wait until the compaction is done
state = admin.getCompactionState(table.getName()); state = admin.getCompactionState(table);
while (state != CompactionState.NONE && curt < endt) { while (state != CompactionState.NONE && curt < endt) {
Thread.sleep(10); Thread.sleep(10);
state = admin.getCompactionState(table.getName()); state = admin.getCompactionState(table);
} }
// Now, compaction should be done. // Now, compaction should be done.
assertEquals(CompactionState.NONE, state); assertEquals(CompactionState.NONE, state);

View File

@ -208,7 +208,7 @@ public class TestEncryptionKeyRotation {
TEST_UTIL.getHBaseAdmin().createTable(htd); TEST_UTIL.getHBaseAdmin().createTable(htd);
TEST_UTIL.waitTableAvailable(htd.getName(), 5000); TEST_UTIL.waitTableAvailable(htd.getName(), 5000);
// Create a store file // Create a store file
Table table = new HTable(conf, htd.getTableName()); Table table = TEST_UTIL.getConnection().getTable(htd.getTableName());
try { try {
table.put(new Put(Bytes.toBytes("testrow")) table.put(new Put(Bytes.toBytes("testrow"))
.add(hcd.getName(), Bytes.toBytes("q"), Bytes.toBytes("value"))); .add(hcd.getName(), Bytes.toBytes("q"), Bytes.toBytes("value")));

View File

@ -103,7 +103,7 @@ public class TestEncryptionRandomKeying {
TEST_UTIL.waitTableAvailable(htd.getName(), 5000); TEST_UTIL.waitTableAvailable(htd.getName(), 5000);
// Create a store file // Create a store file
Table table = new HTable(conf, htd.getTableName()); Table table = TEST_UTIL.getConnection().getTable(htd.getTableName());
try { try {
table.put(new Put(Bytes.toBytes("testrow")) table.put(new Put(Bytes.toBytes("testrow"))
.add(hcd.getName(), Bytes.toBytes("q"), Bytes.toBytes("value"))); .add(hcd.getName(), Bytes.toBytes("q"), Bytes.toBytes("value")));

View File

@ -331,7 +331,7 @@ public class TestEndToEndSplitTransaction {
HTable table = null; HTable table = null;
try { try {
//HTable.getStartEndKeys() //HTable.getStartEndKeys()
table = new HTable(conf, tableName); table = (HTable) connection.getTable(tableName);
Pair<byte[][], byte[][]> keys = table.getStartEndKeys(); Pair<byte[][], byte[][]> keys = table.getStartEndKeys();
verifyStartEndKeys(keys); verifyStartEndKeys(keys);
@ -433,7 +433,8 @@ public class TestEndToEndSplitTransaction {
long start = System.currentTimeMillis(); long start = System.currentTimeMillis();
log("blocking until region is split:" + Bytes.toStringBinary(regionName)); log("blocking until region is split:" + Bytes.toStringBinary(regionName));
HRegionInfo daughterA = null, daughterB = null; 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 { try {
Result result = null; Result result = null;
@ -476,6 +477,7 @@ public class TestEndToEndSplitTransaction {
} }
} finally { } finally {
IOUtils.closeQuietly(metaTable); IOUtils.closeQuietly(metaTable);
IOUtils.closeQuietly(connection);
} }
} }
@ -505,7 +507,8 @@ public class TestEndToEndSplitTransaction {
throws IOException, InterruptedException { throws IOException, InterruptedException {
log("blocking until region is opened for reading:" + hri.getRegionNameAsString()); log("blocking until region is opened for reading:" + hri.getRegionNameAsString());
long start = System.currentTimeMillis(); long start = System.currentTimeMillis();
Table table = new HTable(conf, hri.getTable()); Connection connection = ConnectionFactory.createConnection(conf);
Table table = connection.getTable(hri.getTable());
try { try {
byte [] row = hri.getStartKey(); byte [] row = hri.getStartKey();
@ -523,7 +526,7 @@ public class TestEndToEndSplitTransaction {
} }
} finally { } finally {
IOUtils.closeQuietly(table); IOUtils.closeQuietly(table);
IOUtils.closeQuietly(connection);
} }
} }
} }

View File

@ -42,6 +42,7 @@ import org.apache.hadoop.hbase.HColumnDescriptor;
import org.apache.hadoop.hbase.HConstants; import org.apache.hadoop.hbase.HConstants;
import org.apache.hadoop.hbase.HTableDescriptor; import org.apache.hadoop.hbase.HTableDescriptor;
import org.apache.hadoop.hbase.KeyValue; 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.MediumTests;
import org.apache.hadoop.hbase.testclassification.RegionServerTests; import org.apache.hadoop.hbase.testclassification.RegionServerTests;
import org.apache.hadoop.hbase.TableName; import org.apache.hadoop.hbase.TableName;
@ -190,7 +191,7 @@ public class TestFSErrorsExposed {
TableName tableName = TableName.valueOf("table"); TableName tableName = TableName.valueOf("table");
byte[] fam = Bytes.toBytes("fam"); byte[] fam = Bytes.toBytes("fam");
Admin admin = new HBaseAdmin(util.getConfiguration()); Admin admin = util.getHBaseAdmin();
HTableDescriptor desc = new HTableDescriptor(tableName); HTableDescriptor desc = new HTableDescriptor(tableName);
desc.addFamily(new HColumnDescriptor(fam) desc.addFamily(new HColumnDescriptor(fam)
.setMaxVersions(1) .setMaxVersions(1)
@ -201,7 +202,7 @@ public class TestFSErrorsExposed {
util.getConfiguration().setInt(HConstants.HBASE_CLIENT_RETRIES_NUMBER, 1); util.getConfiguration().setInt(HConstants.HBASE_CLIENT_RETRIES_NUMBER, 1);
// Make a new Configuration so it makes a new connection that has the // 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. // 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 // Load some data
util.loadTable(table, fam, false); util.loadTable(table, fam, false);

View File

@ -76,7 +76,7 @@ public class TestHRegionOnCluster {
// Put data: r1->v1 // Put data: r1->v1
Log.info("Loading r1 to v1 into " + TABLENAME); 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); putDataAndVerify(table, "r1", FAMILY, "v1", 1);
TEST_UTIL.waitUntilAllRegionsAssigned(table.getName()); TEST_UTIL.waitUntilAllRegionsAssigned(table.getName());

Some files were not shown because too many files have changed in this diff Show More