HBASE-12796 Clean up HTable and HBaseAdmin deprecated constructor usage (Jurriaan Mous)
This commit is contained in:
parent
645fbd7d87
commit
9246af8dac
|
@ -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);
|
||||
|
|
|
@ -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();
|
||||
|
|
|
@ -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();
|
||||
}
|
||||
}
|
||||
|
||||
|
|
|
@ -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;
|
||||
}
|
||||
|
||||
|
|
|
@ -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++) {
|
||||
|
|
|
@ -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);
|
||||
|
|
|
@ -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();
|
||||
}
|
||||
}
|
||||
|
|
|
@ -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);
|
||||
}
|
||||
}
|
||||
|
|
|
@ -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()) {
|
||||
|
|
|
@ -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 {
|
||||
|
|
|
@ -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;
|
||||
}
|
||||
|
||||
|
|
|
@ -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;
|
||||
|
|
|
@ -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
|
||||
|
|
|
@ -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);
|
||||
|
|
|
@ -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);
|
||||
|
|
|
@ -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);
|
||||
|
|
|
@ -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();
|
||||
|
|
|
@ -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++) {
|
||||
|
|
|
@ -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 {
|
||||
|
|
|
@ -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();
|
||||
}
|
||||
}
|
|
@ -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);
|
||||
}
|
||||
}
|
||||
}
|
||||
|
|
|
@ -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));
|
||||
}
|
||||
|
|
|
@ -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;
|
||||
}
|
||||
|
||||
|
|
|
@ -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;
|
||||
}
|
||||
}
|
||||
|
|
|
@ -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
|
||||
|
|
|
@ -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();
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
|
|
|
@ -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+"=...");
|
||||
|
|
|
@ -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;
|
||||
|
|
|
@ -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);
|
||||
|
|
|
@ -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.");
|
||||
|
|
|
@ -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);
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
|
|
|
@ -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;
|
||||
|
|
|
@ -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) {
|
||||
|
|
|
@ -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();
|
||||
}
|
||||
}
|
||||
|
|
|
@ -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());
|
||||
|
|
|
@ -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();
|
||||
|
|
|
@ -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 {
|
||||
|
|
|
@ -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="),
|
||||
|
|
|
@ -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],
|
||||
|
|
|
@ -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);
|
||||
|
|
|
@ -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++) {
|
||||
|
|
|
@ -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
|
||||
}
|
||||
|
|
|
@ -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);
|
||||
|
|
|
@ -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());
|
||||
}
|
||||
|
|
|
@ -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();
|
||||
}
|
||||
}
|
||||
}
|
||||
|
|
|
@ -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);
|
||||
|
|
|
@ -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();
|
||||
|
|
|
@ -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);
|
||||
|
|
|
@ -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) {
|
||||
|
|
|
@ -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());
|
||||
|
|
|
@ -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 {
|
||||
|
|
|
@ -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);
|
||||
|
|
|
@ -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 {
|
||||
|
|
|
@ -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");
|
||||
|
||||
}
|
||||
|
|
|
@ -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(
|
||||
|
|
|
@ -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:");
|
||||
|
|
|
@ -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()));
|
||||
|
|
|
@ -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());
|
||||
|
|
|
@ -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);
|
||||
|
|
|
@ -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>(
|
||||
|
|
|
@ -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]);
|
||||
|
|
|
@ -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();
|
||||
|
|
|
@ -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);
|
||||
|
|
|
@ -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);
|
||||
|
|
|
@ -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);
|
||||
|
|
|
@ -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());
|
||||
|
|
|
@ -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);
|
||||
|
|
|
@ -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();
|
||||
}
|
||||
|
||||
}
|
||||
|
|
|
@ -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
|
||||
|
|
|
@ -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();
|
||||
}
|
||||
|
||||
|
|
|
@ -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);
|
||||
|
||||
|
|
|
@ -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);
|
||||
}
|
||||
}
|
||||
|
||||
|
|
|
@ -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);
|
||||
|
|
|
@ -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");
|
||||
|
|
|
@ -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());
|
||||
|
|
|
@ -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 {
|
||||
|
|
|
@ -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" };
|
||||
|
|
|
@ -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);
|
||||
}
|
||||
|
||||
|
|
|
@ -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);
|
||||
|
|
|
@ -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);
|
||||
}
|
||||
|
||||
|
|
|
@ -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 {
|
||||
|
|
|
@ -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;
|
||||
|
|
|
@ -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);
|
||||
|
|
|
@ -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");
|
||||
|
||||
|
|
|
@ -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);
|
||||
|
|
|
@ -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();
|
||||
|
|
|
@ -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);
|
||||
|
|
|
@ -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);
|
||||
|
|
|
@ -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,
|
||||
|
|
|
@ -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");
|
||||
|
|
|
@ -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);
|
||||
|
|
|
@ -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) {
|
||||
|
|
|
@ -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);
|
||||
|
|
|
@ -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);
|
||||
|
|
|
@ -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);
|
||||
|
|
|
@ -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")));
|
||||
|
|
|
@ -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")));
|
||||
|
|
|
@ -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);
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
|
|
|
@ -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);
|
||||
|
|
|
@ -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
Loading…
Reference in New Issue