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.
if (connection instanceof ClusterConnection) {
if (((ClusterConnection) connection).isManaged()) {
return new HTable(TableName.META_TABLE_NAME, (ClusterConnection) connection);
return new HTable(TableName.META_TABLE_NAME, connection);
}
}
return connection.getTable(TableName.META_TABLE_NAME);

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

View File

@ -145,7 +145,8 @@ public class TestClientNoCluster extends Configured implements Tool {
Configuration localConfig = HBaseConfiguration.create(this.conf);
// This override mocks up our exists/get call to throw a RegionServerStoppedException.
localConfig.set("hbase.client.connection.impl", RpcTimeoutConnection.class.getName());
Table table = new HTable(localConfig, TableName.META_TABLE_NAME);
Connection connection = ConnectionFactory.createConnection(localConfig);
Table table = connection.getTable(TableName.META_TABLE_NAME);
Throwable t = null;
LOG.info("Start");
try {
@ -161,6 +162,7 @@ public class TestClientNoCluster extends Configured implements Tool {
} finally {
table.close();
}
connection.close();
LOG.info("Stop");
assertTrue(t != null);
}
@ -182,7 +184,8 @@ public class TestClientNoCluster extends Configured implements Tool {
// and it has expired. Otherwise, if this functionality is broke, all retries will be run --
// all ten of them -- and we'll get the RetriesExhaustedException exception.
localConfig.setInt(HConstants.HBASE_CLIENT_META_OPERATION_TIMEOUT, pause - 1);
Table table = new HTable(localConfig, TableName.META_TABLE_NAME);
Connection connection = ConnectionFactory.createConnection(localConfig);
Table table = connection.getTable(TableName.META_TABLE_NAME);
Throwable t = null;
try {
// An exists call turns into a get w/ a flag.
@ -196,6 +199,7 @@ public class TestClientNoCluster extends Configured implements Tool {
fail();
} finally {
table.close();
connection.close();
}
assertTrue(t != null);
}
@ -216,7 +220,8 @@ public class TestClientNoCluster extends Configured implements Tool {
// Go against meta else we will try to find first region for the table on construction which
// means we'll have to do a bunch more mocking. Tests that go against meta only should be
// good for a bit of testing.
Table table = new HTable(this.conf, TableName.META_TABLE_NAME);
Connection connection = ConnectionFactory.createConnection(this.conf);
Table table = connection.getTable(TableName.META_TABLE_NAME);
ResultScanner scanner = table.getScanner(HConstants.CATALOG_FAMILY);
try {
Result result = null;
@ -226,6 +231,7 @@ public class TestClientNoCluster extends Configured implements Tool {
} finally {
scanner.close();
table.close();
connection.close();
}
}
@ -236,7 +242,8 @@ public class TestClientNoCluster extends Configured implements Tool {
// Go against meta else we will try to find first region for the table on construction which
// means we'll have to do a bunch more mocking. Tests that go against meta only should be
// good for a bit of testing.
Table table = new HTable(this.conf, TableName.META_TABLE_NAME);
Connection connection = ConnectionFactory.createConnection(conf);
Table table = connection.getTable(TableName.META_TABLE_NAME);
ResultScanner scanner = table.getScanner(HConstants.CATALOG_FAMILY);
try {
Result result = null;
@ -246,6 +253,7 @@ public class TestClientNoCluster extends Configured implements Tool {
} finally {
scanner.close();
table.close();
connection.close();
}
}

View File

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

View File

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

View File

@ -76,7 +76,7 @@ public class TestZooKeeperScanPolicyObserver {
.setTimeToLive(1);
desc.addFamily(hcd);
TEST_UTIL.getHBaseAdmin().createTable(desc);
Table t = new HTable(new Configuration(TEST_UTIL.getConfiguration()), tableName);
Table t = TEST_UTIL.getConnection().getTable(tableName);
long now = EnvironmentEdgeManager.currentTime();
ZooKeeperWatcher zkw = new ZooKeeperWatcher(TEST_UTIL.getConfiguration(), "test", null);

View File

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

View File

@ -20,40 +20,35 @@ package org.apache.hadoop.hbase.chaos.actions;
import java.io.IOException;
import java.util.Collection;
import java.util.List;
import org.apache.commons.lang.math.RandomUtils;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.hbase.HRegionLocation;
import org.apache.hadoop.hbase.ServerName;
import org.apache.hadoop.hbase.TableName;
import org.apache.hadoop.hbase.client.Connection;
import org.apache.hadoop.hbase.client.HTable;
import org.apache.hadoop.hbase.client.RegionLocator;
import org.apache.hadoop.hbase.client.Table;
/**
* Action that restarts an HRegionServer holding one of the regions of the table.
*/
public class RestartRsHoldingTableAction extends RestartActionBaseAction {
private final String tableName;
private final RegionLocator locator;
public RestartRsHoldingTableAction(long sleepTime, String tableName) {
public RestartRsHoldingTableAction(long sleepTime, RegionLocator locator) {
super(sleepTime);
this.tableName = tableName;
this.locator = locator;
}
@Override
public void perform() throws Exception {
HTable table = null;
try {
LOG.info("Performing action: Restart random RS holding table " + this.tableName);
Configuration conf = context.getHBaseIntegrationTestingUtility().getConfiguration();
table = new HTable(conf, TableName.valueOf(tableName));
} catch (IOException e) {
LOG.debug("Error creating HTable used to get list of region locations.", e);
return;
}
LOG.info("Performing action: Restart random RS holding table " + this.locator.getName());
Collection<ServerName> serverNames = table.getRegionLocations().values();
ServerName[] nameArray = serverNames.toArray(new ServerName[serverNames.size()]);
restartRs(nameArray[RandomUtils.nextInt(nameArray.length)], sleepTime);
List<HRegionLocation> locations = locator.getAllRegionLocations();
restartRs(locations.get(RandomUtils.nextInt(locations.size())).getServerName(), sleepTime);
}
}

View File

@ -144,7 +144,7 @@ public class IntegrationTestImportTsv implements Configurable, Tool {
setCaching(1000);
}};
try {
table = new HTable(getConf(), tableName);
table = util.getConnection().getTable(tableName);
Iterator<Result> resultsIt = table.getScanner(scan).iterator();
Iterator<KeyValue> expectedIt = simple_expected.iterator();
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
// because this table should only have one region we should be good.
restartRSAction = new RestartRsHoldingTableAction(sleepTime, tableName.getNameAsString());
restartRSAction = new RestartRsHoldingTableAction(sleepTime,
util.getConnection().getRegionLocator(tableName));
// Set up the action that will kill the region holding meta.
restartMetaAction = new RestartRsHoldingMetaAction(sleepTime);
@ -478,7 +479,7 @@ public class IntegrationTestMTTR {
public PutCallable(Future<?> f) throws IOException {
super(f);
this.table = new HTable(util.getConfiguration(), tableName);
this.table = util.getConnection().getTable(tableName);
}
@Override
@ -504,7 +505,7 @@ public class IntegrationTestMTTR {
public ScanCallable(Future<?> f) throws IOException {
super(f);
this.table = new HTable(util.getConfiguration(), tableName);
this.table = util.getConnection().getTable(tableName);
}
@Override
@ -545,7 +546,7 @@ public class IntegrationTestMTTR {
protected boolean doAction() throws Exception {
Admin admin = null;
try {
admin = new HBaseAdmin(util.getConfiguration());
admin = util.getHBaseAdmin();
ClusterStatus status = admin.getClusterStatus();
return status != null;
} finally {

View File

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

View File

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

View File

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

View File

@ -124,7 +124,7 @@ public class IntegrationTestSendTraceRequests extends AbstractHBaseTool {
ResultScanner rs = null;
try {
innerScope = Trace.startSpan("Scan", Sampler.ALWAYS);
Table ht = new HTable(util.getConfiguration(), tableName);
Table ht = util.getConnection().getTable(tableName);
Scan s = new Scan();
s.setStartRow(Bytes.toBytes(rowKeyQueue.take()));
s.setBatch(7);
@ -174,7 +174,7 @@ public class IntegrationTestSendTraceRequests extends AbstractHBaseTool {
Table ht = null;
try {
ht = new HTable(util.getConfiguration(), tableName);
ht = util.getConnection().getTable(tableName);
} catch (IOException e) {
e.printStackTrace();
}
@ -234,7 +234,7 @@ public class IntegrationTestSendTraceRequests extends AbstractHBaseTool {
private LinkedBlockingQueue<Long> insertData() throws IOException, InterruptedException {
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];
for (int x = 0; x < 5000; x++) {
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);
assertEquals(response.getCode(), 200);
Table table = new HTable(TEST_UTIL.getConfiguration(), TABLE);
Table table = TEST_UTIL.getConnection().getTable(TABLE);
Get get = new Get(Bytes.toBytes(ROW_1));
get.addColumn(Bytes.toBytes(CFA), Bytes.toBytes("1"));
Result result = table.get(get);

View File

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

View File

@ -17,23 +17,6 @@
*/
package org.apache.hadoop.hbase.rest;
import static org.junit.Assert.assertEquals;
import static org.junit.Assert.assertNotNull;
import java.io.ByteArrayInputStream;
import java.io.IOException;
import java.io.StringWriter;
import java.security.PrivilegedExceptionAction;
import java.util.ArrayList;
import java.util.Iterator;
import java.util.List;
import java.util.Random;
import javax.xml.bind.JAXBContext;
import javax.xml.bind.JAXBException;
import javax.xml.bind.Marshaller;
import javax.xml.bind.Unmarshaller;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.hbase.HBaseTestingUtility;
import org.apache.hadoop.hbase.HColumnDescriptor;
@ -42,7 +25,6 @@ import org.apache.hadoop.hbase.KeyValue;
import org.apache.hadoop.hbase.TableName;
import org.apache.hadoop.hbase.client.Admin;
import org.apache.hadoop.hbase.client.Durability;
import org.apache.hadoop.hbase.client.HTable;
import org.apache.hadoop.hbase.client.Put;
import org.apache.hadoop.hbase.client.Table;
import org.apache.hadoop.hbase.protobuf.generated.VisibilityLabelsProtos.VisibilityLabelsResponse;
@ -69,6 +51,21 @@ import org.junit.BeforeClass;
import org.junit.Test;
import org.junit.experimental.categories.Category;
import javax.xml.bind.JAXBContext;
import javax.xml.bind.JAXBException;
import javax.xml.bind.Marshaller;
import javax.xml.bind.Unmarshaller;
import java.io.ByteArrayInputStream;
import java.io.IOException;
import java.io.StringWriter;
import java.security.PrivilegedExceptionAction;
import java.util.ArrayList;
import java.util.Iterator;
import java.util.List;
import static org.junit.Assert.assertEquals;
import static org.junit.Assert.assertNotNull;
@Category({RestTests.class, MediumTests.class})
public class TestScannersWithLabels {
private static final TableName TABLE = TableName.valueOf("TestScannersWithLabels");
@ -104,7 +101,7 @@ public class TestScannersWithLabels {
+ TOPSECRET));
puts.add(put);
}
try (Table table = new HTable(TEST_UTIL.getConfiguration(), tableName)) {
try (Table table = TEST_UTIL.getConnection().getTable(tableName)) {
table.put(puts);
}
return puts.size();

View File

@ -87,7 +87,7 @@ public class TestTableResource {
HTableDescriptor htd = new HTableDescriptor(TABLE);
htd.addFamily(new HColumnDescriptor(COLUMN_FAMILY));
admin.createTable(htd);
HTable table = new HTable(TEST_UTIL.getConfiguration(), TABLE);
HTable table = (HTable) TEST_UTIL.getConnection().getTable(TABLE);
byte[] k = new byte[3];
byte [][] famAndQf = KeyValue.parseColumn(Bytes.toBytes(COLUMN));
for (byte b1 = 'a'; b1 < 'z'; b1++) {

View File

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

View File

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

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.HTableInterface;
import org.apache.hadoop.hbase.client.HTableWrapper;
import org.apache.hadoop.hbase.util.Bytes;
import org.apache.hadoop.hbase.util.CoprocessorClassLoader;
import org.apache.hadoop.hbase.util.SortedCopyOnWriteSet;
import org.apache.hadoop.hbase.util.VersionInfo;
@ -325,7 +324,7 @@ public abstract class CoprocessorHost<E extends CoprocessorEnvironment> {
final ClassLoader systemClassLoader = this.getClass().getClassLoader();
for (E env : coprocessors) {
ClassLoader cl = env.getInstance().getClass().getClassLoader();
if (cl != systemClassLoader ){
if (cl != systemClassLoader){
//do not include system classloader
externalClassLoaders.add(cl);
}
@ -434,7 +433,7 @@ public abstract class CoprocessorHost<E extends CoprocessorEnvironment> {
} catch (IOException e) {
// nothing can be done here
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.InterfaceStability;
import org.apache.hadoop.fs.Path;
import org.apache.hadoop.hbase.HBaseConfiguration;
import org.apache.hadoop.hbase.TableName;
import org.apache.hadoop.hbase.client.Connection;
import org.apache.hadoop.hbase.client.ConnectionFactory;
import org.apache.hadoop.hbase.client.HTable;
import org.apache.hadoop.hbase.util.Bytes;
import org.apache.hadoop.mapred.FileInputFormat;
@ -58,8 +59,8 @@ public class TableInputFormat extends TableInputFormatBase implements
}
setInputColumns(m_cols);
try {
setHTable(
new HTable(HBaseConfiguration.create(job), TableName.valueOf(tableNames[0].getName())));
Connection connection = ConnectionFactory.createConnection(job);
setHTable((HTable) connection.getTable(TableName.valueOf(tableNames[0].getName())));
} catch (Exception e) {
LOG.error(StringUtils.stringifyException(e));
}

View File

@ -27,6 +27,7 @@ import org.apache.hadoop.hbase.classification.InterfaceStability;
import org.apache.hadoop.hbase.HConstants;
import org.apache.hadoop.hbase.client.HTable;
import org.apache.hadoop.hbase.client.Result;
import org.apache.hadoop.hbase.client.Table;
import org.apache.hadoop.hbase.filter.Filter;
import org.apache.hadoop.hbase.io.ImmutableBytesWritable;
import org.apache.hadoop.mapred.InputFormat;
@ -158,7 +159,7 @@ implements InputFormat<ImmutableBytesWritable, Result> {
/**
* Allows subclasses to get the {@link HTable}.
*/
protected HTable getHTable() {
protected Table getHTable() {
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.classification.InterfaceAudience;
import org.apache.hadoop.hbase.classification.InterfaceStability;
import org.apache.hadoop.hbase.client.Admin;
import org.apache.hadoop.hbase.client.Connection;
import org.apache.hadoop.hbase.client.ConnectionFactory;
import org.apache.hadoop.hbase.client.Scan;
import org.apache.hadoop.hbase.client.Table;
import org.apache.hadoop.hbase.util.Bytes;
import org.apache.hadoop.mapreduce.Job;
import org.apache.hadoop.util.GenericOptionsParser;
@ -156,8 +156,9 @@ public class CopyTable extends Configured implements Tool {
System.out.println("HFiles will be stored at " + this.bulkloadDir);
HFileOutputFormat2.setOutputPath(job, bulkloadDir);
try (Connection conn = ConnectionFactory.createConnection(getConf());
Table htable = conn.getTable(TableName.valueOf(dstTableName))) {
HFileOutputFormat2.configureIncrementalLoadMap(job, htable);
Admin admin = conn.getAdmin()) {
HFileOutputFormat2.configureIncrementalLoadMap(job,
admin.getTableDescriptor((TableName.valueOf(dstTableName))));
}
} else {
TableMapReduceUtil.initTableMapperJob(tableName, scan,
@ -192,7 +193,8 @@ public class CopyTable extends Configured implements Tool {
System.err.println(" versions number of cell versions to copy");
System.err.println(" new.name new table's name");
System.err.println(" peer.adr Address of the peer cluster given in the format");
System.err.println(" hbase.zookeeer.quorum:hbase.zookeeper.client.port:zookeeper.znode.parent");
System.err.println(" hbase.zookeeper.quorum:hbase.zookeeper.client"
+ ".port:zookeeper.znode.parent");
System.err.println(" families comma-separated list of families to copy");
System.err.println(" To copy from cf1 to cf2, give sourceCfName:destCfName. ");
System.err.println(" To keep the same name, just give \"cfName\"");
@ -298,7 +300,7 @@ public class CopyTable extends Configured implements Tool {
if (i == args.length-1) {
tableName = cmd;
} else {
printUsage("Invalid argument '" + cmd + "'" );
printUsage("Invalid argument '" + cmd + "'");
return false;
}
}

View File

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

View File

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

View File

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

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

View File

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

View File

@ -115,7 +115,7 @@ public class WALPlayer extends Configured implements Tool {
@Override
public void setup(Context context) throws IOException {
// 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) {
// 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.");

View File

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

View File

@ -37,12 +37,13 @@ import org.apache.hadoop.hbase.MetaTableAccessor;
import org.apache.hadoop.hbase.TableName;
import org.apache.hadoop.hbase.TableNotDisabledException;
import org.apache.hadoop.hbase.client.Admin;
import org.apache.hadoop.hbase.client.Connection;
import org.apache.hadoop.hbase.client.ConnectionFactory;
import org.apache.hadoop.hbase.client.Delete;
import org.apache.hadoop.hbase.client.HBaseAdmin;
import org.apache.hadoop.hbase.client.HConnectable;
import org.apache.hadoop.hbase.client.HConnection;
import org.apache.hadoop.hbase.client.HConnectionManager;
import org.apache.hadoop.hbase.client.HTable;
import org.apache.hadoop.hbase.client.Result;
import org.apache.hadoop.hbase.client.ResultScanner;
import org.apache.hadoop.hbase.client.Table;
@ -245,7 +246,8 @@ class HMerge {
throws IOException {
super(conf, fs, tableName);
this.tableName = tableName;
this.table = new HTable(conf, TableName.META_TABLE_NAME);
Connection connection = ConnectionFactory.createConnection(conf);
this.table = connection.getTable(TableName.META_TABLE_NAME);
this.metaScanner = table.getScanner(HConstants.CATALOG_FAMILY,
HConstants.REGIONINFO_QUALIFIER);
this.latestRegion = null;

View File

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

View File

@ -17,34 +17,6 @@
*/
package org.apache.hadoop.hbase;
import static org.junit.Assert.assertEquals;
import static org.junit.Assert.assertTrue;
import static org.junit.Assert.fail;
import java.io.File;
import java.io.IOException;
import java.io.OutputStream;
import java.lang.reflect.Field;
import java.lang.reflect.Modifier;
import java.net.InetAddress;
import java.net.ServerSocket;
import java.net.Socket;
import java.net.UnknownHostException;
import java.security.MessageDigest;
import java.util.ArrayList;
import java.util.Arrays;
import java.util.Collection;
import java.util.Collections;
import java.util.HashSet;
import java.util.List;
import java.util.Map;
import java.util.NavigableSet;
import java.util.Random;
import java.util.Set;
import java.util.TreeSet;
import java.util.UUID;
import java.util.concurrent.TimeUnit;
import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
import org.apache.commons.logging.impl.Jdk14Logger;
@ -90,7 +62,6 @@ import org.apache.hadoop.hbase.regionserver.HStore;
import org.apache.hadoop.hbase.regionserver.InternalScanner;
import org.apache.hadoop.hbase.regionserver.RegionServerServices;
import org.apache.hadoop.hbase.regionserver.RegionServerStoppedException;
import org.apache.hadoop.hbase.wal.WAL;
import org.apache.hadoop.hbase.security.User;
import org.apache.hadoop.hbase.tool.Canary;
import org.apache.hadoop.hbase.util.Bytes;
@ -103,6 +74,7 @@ import org.apache.hadoop.hbase.util.Pair;
import org.apache.hadoop.hbase.util.RegionSplitter;
import org.apache.hadoop.hbase.util.RetryCounter;
import org.apache.hadoop.hbase.util.Threads;
import org.apache.hadoop.hbase.wal.WAL;
import org.apache.hadoop.hbase.zookeeper.EmptyWatcher;
import org.apache.hadoop.hbase.zookeeper.MiniZooKeeperCluster;
import org.apache.hadoop.hbase.zookeeper.ZKConfig;
@ -118,6 +90,34 @@ import org.apache.zookeeper.WatchedEvent;
import org.apache.zookeeper.ZooKeeper;
import org.apache.zookeeper.ZooKeeper.States;
import java.io.File;
import java.io.IOException;
import java.io.OutputStream;
import java.lang.reflect.Field;
import java.lang.reflect.Modifier;
import java.net.InetAddress;
import java.net.ServerSocket;
import java.net.Socket;
import java.net.UnknownHostException;
import java.security.MessageDigest;
import java.util.ArrayList;
import java.util.Arrays;
import java.util.Collection;
import java.util.Collections;
import java.util.HashSet;
import java.util.List;
import java.util.Map;
import java.util.NavigableSet;
import java.util.Random;
import java.util.Set;
import java.util.TreeSet;
import java.util.UUID;
import java.util.concurrent.TimeUnit;
import static org.junit.Assert.assertEquals;
import static org.junit.Assert.assertTrue;
import static org.junit.Assert.fail;
/**
* Facility for testing HBase. Replacement for
* old HBaseTestCase and HBaseClusterTestCase functionality.
@ -976,7 +976,7 @@ public class HBaseTestingUtility extends HBaseCommonTestingUtility {
this.hbaseCluster =
new MiniHBaseCluster(c, numMasters, numSlaves, masterClass, regionserverClass);
// Don't leave here till we've done a successful scan of the hbase:meta
Table t = new HTable(c, TableName.META_TABLE_NAME);
Table t = getConnection().getTable(TableName.META_TABLE_NAME);
ResultScanner s = t.getScanner(new Scan());
while (s.next() != null) {
continue;
@ -996,6 +996,10 @@ public class HBaseTestingUtility extends HBaseCommonTestingUtility {
* @throws IOException
*/
public void restartHBaseCluster(int servers) throws IOException, InterruptedException {
if(connection != null){
connection.close();
connection = null;
}
this.hbaseCluster = new MiniHBaseCluster(this.conf, servers);
// Don't leave here till we've done a successful scan of the hbase:meta
Table t = new HTable(new Configuration(this.conf), TableName.META_TABLE_NAME);
@ -1276,7 +1280,7 @@ public class HBaseTestingUtility extends HBaseCommonTestingUtility {
getHBaseAdmin().createTable(desc, startKey, endKey, numRegions);
// HBaseAdmin only waits for regions to appear in hbase:meta we should wait until they are assigned
waitUntilAllRegionsAssigned(tableName);
return new HTable(getConfiguration(), tableName);
return (HTable) getConnection().getTable(tableName);
}
/**
@ -1315,7 +1319,7 @@ public class HBaseTestingUtility extends HBaseCommonTestingUtility {
getHBaseAdmin().createTable(htd, splitRows);
// HBaseAdmin only waits for regions to appear in hbase:meta we should wait until they are assigned
waitUntilAllRegionsAssigned(htd.getTableName());
return new HTable(getConfiguration(), htd.getTableName());
return (HTable) getConnection().getTable(htd.getTableName());
}
/**
@ -1353,7 +1357,7 @@ public class HBaseTestingUtility extends HBaseCommonTestingUtility {
desc.addFamily(hcd);
}
getHBaseAdmin().createTable(desc);
return new HTable(c, desc.getTableName());
return (HTable) getConnection().getTable(desc.getTableName());
}
/**
@ -1377,7 +1381,7 @@ public class HBaseTestingUtility extends HBaseCommonTestingUtility {
getHBaseAdmin().createTable(desc);
// HBaseAdmin only waits for regions to appear in hbase:meta we should wait until they are assigned
waitUntilAllRegionsAssigned(tableName);
return new HTable(c, tableName);
return (HTable) getConnection().getTable(tableName);
}
/**
@ -1399,7 +1403,7 @@ public class HBaseTestingUtility extends HBaseCommonTestingUtility {
desc.addFamily(hcd);
}
getHBaseAdmin().createTable(desc);
return new HTable(c, desc.getTableName());
return (HTable) getConnection().getTable(desc.getTableName());
}
/**
@ -1461,7 +1465,7 @@ public class HBaseTestingUtility extends HBaseCommonTestingUtility {
getHBaseAdmin().createTable(desc);
// HBaseAdmin only waits for regions to appear in hbase:meta we should wait until they are assigned
waitUntilAllRegionsAssigned(tableName);
return new HTable(new Configuration(getConfiguration()), tableName);
return (HTable) getConnection().getTable(tableName);
}
/**
@ -1498,7 +1502,7 @@ public class HBaseTestingUtility extends HBaseCommonTestingUtility {
getHBaseAdmin().createTable(desc);
// HBaseAdmin only waits for regions to appear in hbase:meta we should wait until they are assigned
waitUntilAllRegionsAssigned(tableName);
return new HTable(new Configuration(getConfiguration()), tableName);
return (HTable) getConnection().getTable(tableName);
}
/**
@ -1537,7 +1541,7 @@ public class HBaseTestingUtility extends HBaseCommonTestingUtility {
getHBaseAdmin().createTable(desc);
// HBaseAdmin only waits for regions to appear in hbase:meta we should wait until they are assigned
waitUntilAllRegionsAssigned(tableName);
return new HTable(new Configuration(getConfiguration()), tableName);
return (HTable) getConnection().getTable(tableName);
}
/**
@ -1569,7 +1573,7 @@ public class HBaseTestingUtility extends HBaseCommonTestingUtility {
getHBaseAdmin().createTable(desc, splitRows);
// HBaseAdmin only waits for regions to appear in hbase:meta we should wait until they are assigned
waitUntilAllRegionsAssigned(tableName);
return new HTable(getConfiguration(), tableName);
return (HTable) getConnection().getTable(tableName);
}
/**
@ -1590,7 +1594,7 @@ public class HBaseTestingUtility extends HBaseCommonTestingUtility {
getHBaseAdmin().createTable(desc, splitRows);
// HBaseAdmin only waits for regions to appear in hbase:meta we should wait until they are assigned
waitUntilAllRegionsAssigned(desc.getTableName());
return new HTable(getConfiguration(), desc.getTableName());
return (HTable) getConnection().getTable(desc.getTableName());
}
/**
@ -1809,7 +1813,7 @@ public class HBaseTestingUtility extends HBaseCommonTestingUtility {
* @throws IOException
*/
public HTable deleteTableData(TableName tableName) throws IOException {
HTable table = new HTable(getConfiguration(), tableName);
HTable table = (HTable) getConnection().getTable(tableName);
Scan scan = new Scan();
ResultScanner resScan = table.getScanner(scan);
for(Result res : resScan) {
@ -1831,7 +1835,7 @@ public class HBaseTestingUtility extends HBaseCommonTestingUtility {
public HTable truncateTable(final TableName tableName, final boolean preserveRegions) throws IOException {
Admin admin = getHBaseAdmin();
admin.truncateTable(tableName, preserveRegions);
return new HTable(getConfiguration(), tableName);
return (HTable) getConnection().getTable(tableName);
}
/**
@ -2271,7 +2275,7 @@ public class HBaseTestingUtility extends HBaseCommonTestingUtility {
public List<HRegionInfo> createMultiRegionsInMeta(final Configuration conf,
final HTableDescriptor htd, byte [][] startKeys)
throws IOException {
Table meta = new HTable(conf, TableName.META_TABLE_NAME);
Table meta = (HTable) getConnection().getTable(TableName.META_TABLE_NAME);
Arrays.sort(startKeys, Bytes.BYTES_COMPARATOR);
List<HRegionInfo> newRegions = new ArrayList<HRegionInfo>(startKeys.length);
// add custom ones
@ -2294,7 +2298,7 @@ public class HBaseTestingUtility extends HBaseCommonTestingUtility {
*/
public List<byte[]> getMetaTableRows() throws IOException {
// TODO: Redo using MetaTableAccessor class
Table t = new HTable(new Configuration(this.conf), TableName.META_TABLE_NAME);
Table t = (HTable) getConnection().getTable(TableName.META_TABLE_NAME);
List<byte[]> rows = new ArrayList<byte[]>();
ResultScanner s = t.getScanner(new Scan());
for (Result result : s) {
@ -2314,7 +2318,7 @@ public class HBaseTestingUtility extends HBaseCommonTestingUtility {
*/
public List<byte[]> getMetaTableRows(TableName tableName) throws IOException {
// TODO: Redo using MetaTableAccessor.
Table t = new HTable(new Configuration(this.conf), TableName.META_TABLE_NAME);
Table t = getConnection().getTable(TableName.META_TABLE_NAME);
List<byte[]> rows = new ArrayList<byte[]>();
ResultScanner s = t.getScanner(new Scan());
for (Result result : s) {
@ -2631,7 +2635,7 @@ public class HBaseTestingUtility extends HBaseCommonTestingUtility {
monitor.close();
if (checkStatus) {
new HTable(new Configuration(conf), TableName.META_TABLE_NAME).close();
getConnection().getTable(TableName.META_TABLE_NAME).close();
}
}
@ -3140,7 +3144,7 @@ public class HBaseTestingUtility extends HBaseCommonTestingUtility {
*/
public void waitUntilAllRegionsAssigned(final TableName tableName, final long timeout)
throws IOException {
final Table meta = new HTable(getConfiguration(), TableName.META_TABLE_NAME);
final Table meta = getConnection().getTable(TableName.META_TABLE_NAME);
try {
waitFor(timeout, 200, true, new Predicate<IOException>() {
@Override
@ -3548,9 +3552,8 @@ public class HBaseTestingUtility extends HBaseCommonTestingUtility {
return totalNumberOfRegions;
}
public static int getMetaRSPort(Configuration conf) throws IOException {
try (Connection c = ConnectionFactory.createConnection();
RegionLocator locator = c.getRegionLocator(TableName.META_TABLE_NAME)) {
public static int getMetaRSPort(Connection connection) throws IOException {
try (RegionLocator locator = connection.getRegionLocator(TableName.META_TABLE_NAME)) {
return locator.getRegionLocation(Bytes.toBytes("")).getPort();
}
}

View File

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

View File

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

View File

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

View File

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

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

View File

@ -252,7 +252,7 @@ public class TestNamespace {
}
//sanity check try to write and read from table
Table table = new HTable(TEST_UTIL.getConfiguration(), desc.getTableName());
Table table = TEST_UTIL.getConnection().getTable(desc.getTableName());
Put p = new Put(Bytes.toBytes("row1"));
p.add(Bytes.toBytes("my_cf"),Bytes.toBytes("my_col"),Bytes.toBytes("value1"));
table.put(p);

View File

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

View File

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

View File

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

View File

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

View File

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

View File

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

View File

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

View File

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

View File

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

View File

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

View File

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

View File

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

View File

@ -131,7 +131,8 @@ public class TestRpcControllerFactory {
// change one of the connection properties so we get a new HConnection with our configuration
conf.setInt(HConstants.HBASE_RPC_TIMEOUT_KEY, HConstants.DEFAULT_HBASE_RPC_TIMEOUT + 1);
Table table = new HTable(conf, name);
Connection connection = ConnectionFactory.createConnection(conf);
Table table = connection.getTable(name);
table.setAutoFlushTo(false);
byte[] row = Bytes.toBytes("row");
Put p = new Put(row);
@ -187,6 +188,7 @@ public class TestRpcControllerFactory {
counter = doScan(table, scanInfo, counter);
table.close();
connection.close();
}
int doScan(Table table, Scan scan, int expectedCount) throws IOException {

View File

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

View File

@ -18,19 +18,17 @@
package org.apache.hadoop.hbase.client;
import java.util.List;
import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.fs.FileSystem;
import org.apache.hadoop.fs.Path;
import org.apache.hadoop.hbase.TableName;
import org.apache.hadoop.hbase.HBaseTestingUtility;
import org.apache.hadoop.hbase.HColumnDescriptor;
import org.apache.hadoop.hbase.HConstants;
import org.apache.hadoop.hbase.HRegionInfo;
import org.apache.hadoop.hbase.HTableDescriptor;
import org.apache.hadoop.hbase.TableName;
import org.apache.hadoop.hbase.master.snapshot.SnapshotManager;
import org.apache.hadoop.hbase.regionserver.ConstantSizeRegionSplitPolicy;
import org.apache.hadoop.hbase.snapshot.SnapshotTestingUtils;
@ -45,6 +43,8 @@ import org.junit.BeforeClass;
import org.junit.Test;
import org.junit.experimental.categories.Category;
import java.util.List;
/**
* Test to verify that the cloned table is independent of the table from which it was cloned
*/
@ -211,7 +211,7 @@ public class TestSnapshotCloneIndependence {
TableName cloneTableName = TableName.valueOf("test-clone-" + localTableName);
admin.cloneSnapshot(snapshotName, cloneTableName);
try (Table clonedTable = new HTable(UTIL.getConfiguration(), cloneTableName)){
try (Table clonedTable = UTIL.getConnection().getTable(cloneTableName)) {
final int clonedTableRowCount = UTIL.countRows(clonedTable);
Assert.assertEquals(

View File

@ -149,7 +149,7 @@ public class TestSnapshotFromClient {
SnapshotTestingUtils.assertNoSnapshots(admin);
// put some stuff in the table
HTable table = new HTable(UTIL.getConfiguration(), TABLE_NAME);
Table table = UTIL.getConnection().getTable(TABLE_NAME);
UTIL.loadTable(table, TEST_FAM);
table.close();
@ -185,7 +185,7 @@ public class TestSnapshotFromClient {
SnapshotTestingUtils.assertNoSnapshots(admin);
// put some stuff in the table
HTable table = new HTable(UTIL.getConfiguration(), TABLE_NAME);
Table table = UTIL.getConnection().getTable(TABLE_NAME);
UTIL.loadTable(table, TEST_FAM, false);
LOG.debug("FS state before disable:");

View File

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

View File

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

View File

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

View File

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

View File

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

View File

@ -18,23 +18,12 @@
*/
package org.apache.hadoop.hbase.coprocessor;
import static org.junit.Assert.assertEquals;
import static org.junit.Assert.assertTrue;
import java.io.IOException;
import java.util.Collections;
import java.util.concurrent.ExecutorService;
import java.util.concurrent.SynchronousQueue;
import java.util.concurrent.ThreadPoolExecutor;
import java.util.concurrent.TimeUnit;
import org.apache.hadoop.hbase.HBaseTestingUtility;
import org.apache.hadoop.hbase.HColumnDescriptor;
import org.apache.hadoop.hbase.HTableDescriptor;
import org.apache.hadoop.hbase.TableName;
import org.apache.hadoop.hbase.client.Admin;
import org.apache.hadoop.hbase.client.Durability;
import org.apache.hadoop.hbase.client.HTable;
import org.apache.hadoop.hbase.client.Put;
import org.apache.hadoop.hbase.client.Result;
import org.apache.hadoop.hbase.client.ResultScanner;
@ -50,6 +39,16 @@ import org.junit.BeforeClass;
import org.junit.Test;
import org.junit.experimental.categories.Category;
import java.io.IOException;
import java.util.Collections;
import java.util.concurrent.ExecutorService;
import java.util.concurrent.SynchronousQueue;
import java.util.concurrent.ThreadPoolExecutor;
import java.util.concurrent.TimeUnit;
import static org.junit.Assert.assertEquals;
import static org.junit.Assert.assertTrue;
/**
* Test that a coprocessor can open a connection and write to another table, inside a hook.
*/
@ -161,13 +160,13 @@ public class TestOpenTableInCoprocessor {
admin.createTable(primary);
admin.createTable(other);
Table table = new HTable(UTIL.getConfiguration(), TableName.valueOf("primary"));
Table table = UTIL.getConnection().getTable(TableName.valueOf("primary"));
Put p = new Put(new byte[] { 'a' });
p.add(family, null, new byte[] { 'a' });
table.put(p);
table.close();
Table target = new HTable(UTIL.getConfiguration(), otherTable);
Table target = UTIL.getConnection().getTable(otherTable);
assertTrue("Didn't complete update to target table!", completeCheck[0]);
assertEquals("Didn't find inserted row", 1, getKeyValueCount(target));
target.close();

View File

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

View File

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

View File

@ -259,7 +259,7 @@ public class TestRegionObserverScannerOpenHook {
Admin admin = UTIL.getHBaseAdmin();
admin.createTable(desc);
Table table = new HTable(conf, desc.getTableName());
Table table = UTIL.getConnection().getTable(desc.getTableName());
// put a row and flush it to disk
Put put = new Put(ROW);

View File

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

View File

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

View File

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

View File

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

View File

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

View File

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

View File

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

View File

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

View File

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

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

View File

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

View File

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

View File

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

View File

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

View File

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

View File

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

View File

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

View File

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

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

View File

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

View File

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

View File

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

View File

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

View File

@ -253,7 +253,7 @@ public class TestMasterOperationsForRegionReplicas {
ADMIN.disableTable(table);
// now delete one replica info from all the rows
// this is to make the meta appear to be only partially updated
Table metaTable = new HTable(TableName.META_TABLE_NAME, ADMIN.getConnection());
Table metaTable = ADMIN.getConnection().getTable(TableName.META_TABLE_NAME);
for (byte[] row : tableRows) {
Delete deleteOneReplicaLocation = new Delete(row);
deleteOneReplicaLocation.deleteColumns(HConstants.CATALOG_FAMILY,

View File

@ -99,7 +99,7 @@ public class TestMasterRestartAfterDisablingTable {
TableState.State.DISABLING));
log("Enabling table\n");
// Need a new Admin, the previous one is on the old master
Admin admin = new HBaseAdmin(TEST_UTIL.getConfiguration());
Admin admin = TEST_UTIL.getHBaseAdmin();
admin.enableTable(table);
admin.close();
log("Waiting for no more RIT\n");

View File

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

View File

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

View File

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

View File

@ -36,6 +36,7 @@ import org.apache.hadoop.hbase.HColumnDescriptor;
import org.apache.hadoop.hbase.HTableDescriptor;
import org.apache.hadoop.hbase.client.Admin;
import org.apache.hadoop.hbase.client.HTable;
import org.apache.hadoop.hbase.client.Table;
import org.apache.hadoop.hbase.testclassification.LargeTests;
import org.apache.hadoop.hbase.testclassification.MasterTests;
import org.apache.hadoop.hbase.util.Bytes;
@ -69,7 +70,7 @@ public class TestTableDeleteFamilyHandler {
// Create a table of three families. This will assign a region.
TEST_UTIL.createTable(TABLENAME, FAMILIES);
HTable t = new HTable(TEST_UTIL.getConfiguration(), TABLENAME);
Table t = TEST_UTIL.getConnection().getTable(TABLENAME);
while(TEST_UTIL.getMiniHBaseCluster().getMaster().getAssignmentManager()
.getRegionStates().getRegionsInTransition().size() > 0) {
Thread.sleep(100);

View File

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

View File

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

View File

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

View File

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

View File

@ -42,6 +42,7 @@ import org.apache.hadoop.hbase.HColumnDescriptor;
import org.apache.hadoop.hbase.HConstants;
import org.apache.hadoop.hbase.HTableDescriptor;
import org.apache.hadoop.hbase.KeyValue;
import org.apache.hadoop.hbase.client.Table;
import org.apache.hadoop.hbase.testclassification.MediumTests;
import org.apache.hadoop.hbase.testclassification.RegionServerTests;
import org.apache.hadoop.hbase.TableName;
@ -190,7 +191,7 @@ public class TestFSErrorsExposed {
TableName tableName = TableName.valueOf("table");
byte[] fam = Bytes.toBytes("fam");
Admin admin = new HBaseAdmin(util.getConfiguration());
Admin admin = util.getHBaseAdmin();
HTableDescriptor desc = new HTableDescriptor(tableName);
desc.addFamily(new HColumnDescriptor(fam)
.setMaxVersions(1)
@ -201,7 +202,7 @@ public class TestFSErrorsExposed {
util.getConfiguration().setInt(HConstants.HBASE_CLIENT_RETRIES_NUMBER, 1);
// Make a new Configuration so it makes a new connection that has the
// above configuration on it; else we use the old one w/ 10 as default.
HTable table = new HTable(new Configuration(util.getConfiguration()), tableName);
Table table = util.getConnection().getTable(tableName);
// Load some data
util.loadTable(table, fam, false);

View File

@ -76,7 +76,7 @@ public class TestHRegionOnCluster {
// Put data: r1->v1
Log.info("Loading r1 to v1 into " + TABLENAME);
HTable table = new HTable(TEST_UTIL.getConfiguration(), TABLENAME);
HTable table = (HTable) TEST_UTIL.getConnection().getTable(TABLENAME);
putDataAndVerify(table, "r1", FAMILY, "v1", 1);
TEST_UTIL.waitUntilAllRegionsAssigned(table.getName());

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