HBASE-11679 Replace HTable with HTableInterface where backwards-compatible (Carter)

This commit is contained in:
Enis Soztutar 2014-09-08 23:39:34 -07:00
parent 71e6ff4377
commit 4995ed8a02
207 changed files with 1226 additions and 1105 deletions

View File

@ -34,6 +34,7 @@ import org.apache.hadoop.hbase.client.RegionReplicaUtil;
import org.apache.hadoop.hbase.client.Result;
import org.apache.hadoop.hbase.client.ResultScanner;
import org.apache.hadoop.hbase.client.Scan;
import org.apache.hadoop.hbase.client.Table;
import org.apache.hadoop.hbase.ipc.CoprocessorRpcChannel;
import org.apache.hadoop.hbase.protobuf.ProtobufUtil;
import org.apache.hadoop.hbase.protobuf.generated.ClientProtos;
@ -174,7 +175,7 @@ public class MetaTableAccessor {
* @throws IOException
* @SuppressWarnings("deprecation")
*/
private static HTable getHTable(final HConnection hConnection,
private static Table getHTable(final HConnection hConnection,
final TableName tableName)
throws IOException {
// We used to pass whole CatalogTracker in here, now we just pass in HConnection
@ -190,7 +191,7 @@ public class MetaTableAccessor {
* @return An {@link HTable} for <code>hbase:meta</code>
* @throws IOException
*/
static HTable getMetaHTable(final HConnection hConnection)
static Table getMetaHTable(final HConnection hConnection)
throws IOException {
return getHTable(hConnection, TableName.META_TABLE_NAME);
}
@ -200,7 +201,7 @@ public class MetaTableAccessor {
* @param g Get to run
* @throws IOException
*/
private static Result get(final HTable t, final Get g) throws IOException {
private static Result get(final Table t, final Get g) throws IOException {
try {
return t.get(g);
} finally {
@ -599,7 +600,7 @@ public class MetaTableAccessor {
scan.setCaching(caching);
}
scan.addFamily(HConstants.CATALOG_FAMILY);
HTable metaTable = getMetaHTable(hConnection);
Table metaTable = getMetaHTable(hConnection);
ResultScanner scanner = null;
try {
scanner = metaTable.getScanner(scan);
@ -948,7 +949,7 @@ public class MetaTableAccessor {
* @param p put to make
* @throws IOException
*/
private static void put(final HTable t, final Put p) throws IOException {
private static void put(final Table t, final Put p) throws IOException {
try {
t.put(p);
} finally {
@ -964,7 +965,7 @@ public class MetaTableAccessor {
*/
public static void putsToMetaTable(final HConnection hConnection, final List<Put> ps)
throws IOException {
HTable t = getMetaHTable(hConnection);
Table t = getMetaHTable(hConnection);
try {
t.put(ps);
} finally {
@ -993,7 +994,7 @@ public class MetaTableAccessor {
*/
public static void deleteFromMetaTable(final HConnection hConnection, final List<Delete> deletes)
throws IOException {
HTable t = getMetaHTable(hConnection);
Table t = getMetaHTable(hConnection);
try {
t.delete(deletes);
} finally {
@ -1036,7 +1037,7 @@ public class MetaTableAccessor {
public static void mutateMetaTable(final HConnection hConnection,
final List<Mutation> mutations)
throws IOException {
HTable t = getMetaHTable(hConnection);
Table t = getMetaHTable(hConnection);
try {
t.batch(mutations);
} catch (InterruptedException e) {
@ -1068,7 +1069,7 @@ public class MetaTableAccessor {
* @param regionInfo region information
* @throws IOException if problem connecting or updating meta
*/
public static void addRegionToMeta(HTable meta, HRegionInfo regionInfo) throws IOException {
public static void addRegionToMeta(Table meta, HRegionInfo regionInfo) throws IOException {
addRegionToMeta(meta, regionInfo, null, null);
}
@ -1085,7 +1086,7 @@ public class MetaTableAccessor {
* @param splitB second split daughter of the parent regionInfo
* @throws IOException if problem connecting or updating meta
*/
public static void addRegionToMeta(HTable meta, HRegionInfo regionInfo,
public static void addRegionToMeta(Table meta, HRegionInfo regionInfo,
HRegionInfo splitA, HRegionInfo splitB) throws IOException {
Put put = makePutFromRegionInfo(regionInfo);
addDaughtersToPut(put, splitA, splitB);
@ -1109,7 +1110,7 @@ public class MetaTableAccessor {
*/
public static void addRegionToMeta(HConnection hConnection, HRegionInfo regionInfo,
HRegionInfo splitA, HRegionInfo splitB) throws IOException {
HTable meta = getMetaHTable(hConnection);
Table meta = getMetaHTable(hConnection);
try {
addRegionToMeta(meta, regionInfo, splitA, splitB);
} finally {
@ -1168,7 +1169,7 @@ public class MetaTableAccessor {
*/
public static void mergeRegions(final HConnection hConnection, HRegionInfo mergedRegion,
HRegionInfo regionA, HRegionInfo regionB, ServerName sn) throws IOException {
HTable meta = getMetaHTable(hConnection);
Table meta = getMetaHTable(hConnection);
try {
HRegionInfo copyOfMerged = new HRegionInfo(mergedRegion);
@ -1208,7 +1209,7 @@ public class MetaTableAccessor {
public static void splitRegion(final HConnection hConnection,
HRegionInfo parent, HRegionInfo splitA, HRegionInfo splitB,
ServerName sn) throws IOException {
HTable meta = getMetaHTable(hConnection);
Table meta = getMetaHTable(hConnection);
try {
HRegionInfo copyOfParent = new HRegionInfo(parent);
copyOfParent.setOffline(true);
@ -1235,7 +1236,7 @@ public class MetaTableAccessor {
/**
* Performs an atomic multi-Mutate operation against the given table.
*/
private static void multiMutate(HTable table, byte[] row, Mutation... mutations)
private static void multiMutate(Table table, byte[] row, Mutation... mutations)
throws IOException {
CoprocessorRpcChannel channel = table.coprocessorService(row);
MultiRowMutationProtos.MutateRowsRequest.Builder mmrBuilder

View File

@ -22,7 +22,6 @@ import java.io.Closeable;
import java.io.IOException;
import java.io.InterruptedIOException;
import java.util.ArrayList;
import java.util.Arrays;
import java.util.Collections;
import java.util.LinkedList;
import java.util.List;
@ -1779,7 +1778,7 @@ public class HTable implements HTableInterface, RegionLocator {
* @throws IOException
*/
public static void main(String[] args) throws IOException {
HTable t = new HTable(HBaseConfiguration.create(), args[0]);
Table t = new HTable(HBaseConfiguration.create(), args[0]);
try {
System.out.println(t.get(new Get(Bytes.toBytes(args[1]))));
} finally {

View File

@ -19,7 +19,6 @@
package org.apache.hadoop.hbase.client;
import org.apache.hadoop.classification.InterfaceAudience;
import org.apache.hadoop.classification.InterfaceStability;
import org.apache.hadoop.hbase.HRegionLocation;
import java.io.IOException;
@ -102,7 +101,7 @@ public class HTableUtil {
}
private static Map<String,List<Put>> createRsPutMap(HTable htable, List<Put> puts) throws IOException {
private static Map<String,List<Put>> createRsPutMap(RegionLocator htable, List<Put> puts) throws IOException {
Map<String, List<Put>> putMap = new HashMap<String, List<Put>>();
for (Put put: puts) {
@ -118,7 +117,7 @@ public class HTableUtil {
return putMap;
}
private static Map<String,List<Row>> createRsRowMap(HTable htable, List<Row> rows) throws IOException {
private static Map<String,List<Row>> createRsRowMap(RegionLocator htable, List<Row> rows) throws IOException {
Map<String, List<Row>> rowMap = new HashMap<String, List<Row>>();
for (Row row: rows) {

View File

@ -40,6 +40,7 @@ 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.Scan;
import org.apache.hadoop.hbase.client.Table;
import org.apache.hadoop.hbase.coprocessor.ColumnInterpreter;
import org.apache.hadoop.hbase.ipc.BlockingRpcCallback;
import org.apache.hadoop.hbase.ipc.ServerRpcController;
@ -101,7 +102,7 @@ public class AggregationClient {
public <R, S, P extends Message, Q extends Message, T extends Message> R max(
final TableName tableName, final ColumnInterpreter<R, S, P, Q, T> ci, final Scan scan)
throws Throwable {
HTable table = null;
Table table = null;
try {
table = new HTable(conf, tableName);
return max(table, ci, scan);
@ -125,7 +126,7 @@ public class AggregationClient {
* & propagated to it.
*/
public <R, S, P extends Message, Q extends Message, T extends Message>
R max(final HTable table, final ColumnInterpreter<R, S, P, Q, T> ci,
R max(final Table table, final ColumnInterpreter<R, S, P, Q, T> ci,
final Scan scan) throws Throwable {
final AggregateRequest requestArg = validateArgAndGetPB(scan, ci, false);
class MaxCallBack implements Batch.Callback<R> {
@ -196,7 +197,7 @@ public class AggregationClient {
public <R, S, P extends Message, Q extends Message, T extends Message> R min(
final TableName tableName, final ColumnInterpreter<R, S, P, Q, T> ci, final Scan scan)
throws Throwable {
HTable table = null;
Table table = null;
try {
table = new HTable(conf, tableName);
return min(table, ci, scan);
@ -218,7 +219,7 @@ public class AggregationClient {
* @throws Throwable
*/
public <R, S, P extends Message, Q extends Message, T extends Message>
R min(final HTable table, final ColumnInterpreter<R, S, P, Q, T> ci,
R min(final Table table, final ColumnInterpreter<R, S, P, Q, T> ci,
final Scan scan) throws Throwable {
final AggregateRequest requestArg = validateArgAndGetPB(scan, ci, false);
class MinCallBack implements Batch.Callback<R> {
@ -276,7 +277,7 @@ public class AggregationClient {
public <R, S, P extends Message, Q extends Message, T extends Message> long rowCount(
final TableName tableName, final ColumnInterpreter<R, S, P, Q, T> ci, final Scan scan)
throws Throwable {
HTable table = null;
Table table = null;
try {
table = new HTable(conf, tableName);
return rowCount(table, ci, scan);
@ -301,7 +302,7 @@ public class AggregationClient {
* @throws Throwable
*/
public <R, S, P extends Message, Q extends Message, T extends Message>
long rowCount(final HTable table,
long rowCount(final Table table,
final ColumnInterpreter<R, S, P, Q, T> ci, final Scan scan) throws Throwable {
final AggregateRequest requestArg = validateArgAndGetPB(scan, ci, true);
class RowNumCallback implements Batch.Callback<Long> {
@ -350,7 +351,7 @@ public class AggregationClient {
public <R, S, P extends Message, Q extends Message, T extends Message> S sum(
final TableName tableName, final ColumnInterpreter<R, S, P, Q, T> ci, final Scan scan)
throws Throwable {
HTable table = null;
Table table = null;
try {
table = new HTable(conf, tableName);
return sum(table, ci, scan);
@ -371,7 +372,7 @@ public class AggregationClient {
* @throws Throwable
*/
public <R, S, P extends Message, Q extends Message, T extends Message>
S sum(final HTable table, final ColumnInterpreter<R, S, P, Q, T> ci,
S sum(final Table table, final ColumnInterpreter<R, S, P, Q, T> ci,
final Scan scan) throws Throwable {
final AggregateRequest requestArg = validateArgAndGetPB(scan, ci, false);
@ -423,7 +424,7 @@ public class AggregationClient {
private <R, S, P extends Message, Q extends Message, T extends Message> Pair<S, Long> getAvgArgs(
final TableName tableName, final ColumnInterpreter<R, S, P, Q, T> ci, final Scan scan)
throws Throwable {
HTable table = null;
Table table = null;
try {
table = new HTable(conf, tableName);
return getAvgArgs(table, ci, scan);
@ -443,7 +444,7 @@ public class AggregationClient {
* @throws Throwable
*/
private <R, S, P extends Message, Q extends Message, T extends Message>
Pair<S, Long> getAvgArgs(final HTable table,
Pair<S, Long> getAvgArgs(final Table table,
final ColumnInterpreter<R, S, P, Q, T> ci, final Scan scan) throws Throwable {
final AggregateRequest requestArg = validateArgAndGetPB(scan, ci, false);
class AvgCallBack implements Batch.Callback<Pair<S, Long>> {
@ -523,7 +524,7 @@ public class AggregationClient {
* @throws Throwable
*/
public <R, S, P extends Message, Q extends Message, T extends Message> double avg(
final HTable table, final ColumnInterpreter<R, S, P, Q, T> ci, Scan scan) throws Throwable {
final Table table, final ColumnInterpreter<R, S, P, Q, T> ci, Scan scan) throws Throwable {
Pair<S, Long> p = getAvgArgs(table, ci, scan);
return ci.divideForAvg(p.getFirst(), p.getSecond());
}
@ -540,7 +541,7 @@ public class AggregationClient {
* @throws Throwable
*/
private <R, S, P extends Message, Q extends Message, T extends Message>
Pair<List<S>, Long> getStdArgs(final HTable table,
Pair<List<S>, Long> getStdArgs(final Table table,
final ColumnInterpreter<R, S, P, Q, T> ci, final Scan scan) throws Throwable {
final AggregateRequest requestArg = validateArgAndGetPB(scan, ci, false);
class StdCallback implements Batch.Callback<Pair<List<S>, Long>> {
@ -614,7 +615,7 @@ public class AggregationClient {
public <R, S, P extends Message, Q extends Message, T extends Message>
double std(final TableName tableName, ColumnInterpreter<R, S, P, Q, T> ci,
Scan scan) throws Throwable {
HTable table = null;
Table table = null;
try {
table = new HTable(conf, tableName);
return std(table, ci, scan);
@ -638,7 +639,7 @@ public class AggregationClient {
* @throws Throwable
*/
public <R, S, P extends Message, Q extends Message, T extends Message> double std(
final HTable table, ColumnInterpreter<R, S, P, Q, T> ci, Scan scan) throws Throwable {
final Table table, ColumnInterpreter<R, S, P, Q, T> ci, Scan scan) throws Throwable {
Pair<List<S>, Long> p = getStdArgs(table, ci, scan);
double res = 0d;
double avg = ci.divideForAvg(p.getFirst().get(0), p.getSecond());
@ -662,7 +663,7 @@ public class AggregationClient {
*/
private <R, S, P extends Message, Q extends Message, T extends Message>
Pair<NavigableMap<byte[], List<S>>, List<S>>
getMedianArgs(final HTable table,
getMedianArgs(final Table table,
final ColumnInterpreter<R, S, P, Q, T> ci, final Scan scan) throws Throwable {
final AggregateRequest requestArg = validateArgAndGetPB(scan, ci, false);
final NavigableMap<byte[], List<S>> map =
@ -727,7 +728,7 @@ public class AggregationClient {
public <R, S, P extends Message, Q extends Message, T extends Message>
R median(final TableName tableName, ColumnInterpreter<R, S, P, Q, T> ci,
Scan scan) throws Throwable {
HTable table = null;
Table table = null;
try {
table = new HTable(conf, tableName);
return median(table, ci, scan);
@ -749,7 +750,7 @@ public class AggregationClient {
* @throws Throwable
*/
public <R, S, P extends Message, Q extends Message, T extends Message>
R median(final HTable table, ColumnInterpreter<R, S, P, Q, T> ci,
R median(final Table table, ColumnInterpreter<R, S, P, Q, T> ci,
Scan scan) throws Throwable {
Pair<NavigableMap<byte[], List<S>>, List<S>> p = getMedianArgs(table, ci, scan);
byte[] startRow = null;

View File

@ -21,12 +21,12 @@ package org.apache.hadoop.hbase.client.coprocessor;
import static org.apache.hadoop.hbase.HConstants.EMPTY_START_ROW;
import static org.apache.hadoop.hbase.HConstants.LAST_ROW;
import org.apache.hadoop.hbase.client.Table;
import org.apache.hadoop.hbase.util.ByteStringer;
import org.apache.hadoop.classification.InterfaceAudience;
import org.apache.hadoop.fs.Path;
import org.apache.hadoop.hbase.TableName;
import org.apache.hadoop.hbase.client.HTable;
import org.apache.hadoop.hbase.ipc.BlockingRpcCallback;
import org.apache.hadoop.hbase.ipc.CoprocessorRpcChannel;
import org.apache.hadoop.hbase.ipc.ServerRpcController;
@ -47,9 +47,9 @@ import java.util.List;
*/
@InterfaceAudience.Private
public class SecureBulkLoadClient {
private HTable table;
private Table table;
public SecureBulkLoadClient(HTable table) {
public SecureBulkLoadClient(Table table) {
this.table = table;
}

View File

@ -32,8 +32,10 @@ import org.apache.hadoop.hbase.MasterNotRunningException;
import org.apache.hadoop.hbase.NamespaceDescriptor;
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.HBaseAdmin;
import org.apache.hadoop.hbase.client.HTable;
import org.apache.hadoop.hbase.client.Table;
import org.apache.hadoop.hbase.client.coprocessor.Batch;
import org.apache.hadoop.hbase.ipc.BlockingRpcCallback;
import org.apache.hadoop.hbase.ipc.CoprocessorRpcChannel;
@ -70,7 +72,7 @@ public class AccessControlClient {
public static GrantResponse grant(Configuration conf, final TableName tableName,
final String userName, final byte[] family, final byte[] qual,
final AccessControlProtos.Permission.Action... actions) throws Throwable {
HTable ht = null;
Table ht = null;
try {
TableName aclTableName =
TableName.valueOf(NamespaceDescriptor.SYSTEM_NAMESPACE_NAME_STR, "acl");
@ -150,7 +152,7 @@ public class AccessControlClient {
public static RevokeResponse revoke(Configuration conf, final String username,
final TableName tableName, final byte[] family, final byte[] qualifier,
final AccessControlProtos.Permission.Action... actions) throws Throwable {
HTable ht = null;
Table ht = null;
try {
TableName aclTableName = TableName.valueOf(NamespaceDescriptor.SYSTEM_NAMESPACE_NAME_STR,
"acl");
@ -211,8 +213,8 @@ public class AccessControlClient {
public static List<UserPermission> getUserPermissions(Configuration conf, String tableRegex)
throws Throwable {
List<UserPermission> permList = new ArrayList<UserPermission>();
HTable ht = null;
HBaseAdmin ha = null;
Table ht = null;
Admin ha = null;
try {
TableName aclTableName =
TableName.valueOf(NamespaceDescriptor.SYSTEM_NAMESPACE_NAME_STR, "acl");

View File

@ -22,6 +22,7 @@ import static org.apache.hadoop.hbase.security.visibility.VisibilityConstants.LA
import java.io.IOException;
import java.util.Map;
import org.apache.hadoop.hbase.client.Table;
import org.apache.hadoop.hbase.util.ByteStringer;
import org.apache.hadoop.classification.InterfaceAudience;
import org.apache.hadoop.classification.InterfaceStability;
@ -72,7 +73,7 @@ public class VisibilityClient {
*/
public static VisibilityLabelsResponse addLabels(Configuration conf, final String[] labels)
throws Throwable {
HTable ht = null;
Table ht = null;
try {
ht = new HTable(conf, LABELS_TABLE_NAME.getName());
Batch.Call<VisibilityLabelsService, VisibilityLabelsResponse> callable =
@ -126,7 +127,7 @@ public class VisibilityClient {
* @throws Throwable
*/
public static GetAuthsResponse getAuths(Configuration conf, final String user) throws Throwable {
HTable ht = null;
Table ht = null;
try {
ht = new HTable(conf, LABELS_TABLE_NAME.getName());
Batch.Call<VisibilityLabelsService, GetAuthsResponse> callable =
@ -168,7 +169,7 @@ public class VisibilityClient {
private static VisibilityLabelsResponse setOrClearAuths(Configuration conf, final String[] auths,
final String user, final boolean setOrClear) throws IOException, ServiceException, Throwable {
HTable ht = null;
Table ht = null;
try {
ht = new HTable(conf, LABELS_TABLE_NAME.getName());
Batch.Call<VisibilityLabelsService, VisibilityLabelsResponse> callable =

View File

@ -150,7 +150,7 @@ 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());
HTable table = new HTable(localConfig, TableName.META_TABLE_NAME);
Table table = new HTable(localConfig, TableName.META_TABLE_NAME);
Throwable t = null;
LOG.info("Start");
try {
@ -187,7 +187,7 @@ 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);
HTable table = new HTable(localConfig, TableName.META_TABLE_NAME);
Table table = new HTable(localConfig, TableName.META_TABLE_NAME);
Throwable t = null;
try {
// An exists call turns into a get w/ a flag.
@ -219,7 +219,7 @@ 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.
HTable table = new HTable(this.conf, TableName.META_TABLE_NAME);
Table table = new HTable(this.conf, TableName.META_TABLE_NAME);
ResultScanner scanner = table.getScanner(HConstants.CATALOG_FAMILY);
try {
Result result = null;
@ -239,7 +239,7 @@ 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.
HTable table = new HTable(this.conf, TableName.META_TABLE_NAME);
Table table = new HTable(this.conf, TableName.META_TABLE_NAME);
ResultScanner scanner = table.getScanner(HConstants.CATALOG_FAMILY);
try {
Result result = null;
@ -700,7 +700,7 @@ public class TestClientNoCluster extends Configured implements Tool {
* @throws IOException
*/
static void cycle(int id, final Configuration c, final HConnection sharedConnection) throws IOException {
HTableInterface table = sharedConnection.getTable(BIG_USER_TABLE);
Table table = sharedConnection.getTable(BIG_USER_TABLE);
table.setAutoFlushTo(false);
long namespaceSpan = c.getLong("hbase.test.namespace.span", 1000000);
long startTime = System.currentTimeMillis();

View File

@ -99,7 +99,7 @@ public class TestSnapshotFromAdmin {
builder.build(), builder.build(), builder.build(), builder.setDone(true).build());
// setup the admin and run the test
HBaseAdmin admin = new HBaseAdmin(mockConnection);
Admin admin = new HBaseAdmin(mockConnection);
String snapshot = "snapshot";
TableName table = TableName.valueOf("table");
// get start time
@ -122,7 +122,7 @@ public class TestSnapshotFromAdmin {
.mock(ConnectionManager.HConnectionImplementation.class);
Configuration conf = HBaseConfiguration.create();
Mockito.when(mockConnection.getConfiguration()).thenReturn(conf);
HBaseAdmin admin = new HBaseAdmin(mockConnection);
Admin admin = new HBaseAdmin(mockConnection);
SnapshotDescription.Builder builder = SnapshotDescription.newBuilder();
// check that invalid snapshot names fail
failSnapshotStart(admin, builder.setName(HConstants.SNAPSHOT_DIR_NAME).build());
@ -152,7 +152,7 @@ public class TestSnapshotFromAdmin {
admin.snapshot(builder.setName("snapshot").setTable("table").build());
}
private void failSnapshotStart(HBaseAdmin admin, SnapshotDescription snapshot) throws IOException {
private void failSnapshotStart(Admin admin, SnapshotDescription snapshot) throws IOException {
try {
admin.snapshot(snapshot);
fail("Snapshot should not have succeed with name:" + snapshot.getName());

View File

@ -37,6 +37,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.client.coprocessor.Batch;
import org.apache.hadoop.hbase.coprocessor.CoprocessorHost;
import org.apache.hadoop.hbase.coprocessor.example.generated.BulkDeleteProtos.BulkDeleteRequest;
@ -78,7 +79,7 @@ public class TestBulkDeleteProtocol {
// @Ignore @Test
public void testBulkDeleteEndpoint() throws Throwable {
byte[] tableName = Bytes.toBytes("testBulkDeleteEndpoint");
HTable ht = createTable(tableName);
Table ht = createTable(tableName);
List<Put> puts = new ArrayList<Put>(100);
for (int j = 0; j < 100; j++) {
byte[] rowkey = Bytes.toBytes(j);
@ -102,7 +103,7 @@ public class TestBulkDeleteProtocol {
throws Throwable {
byte[] tableName = Bytes
.toBytes("testBulkDeleteEndpointWhenRowBatchSizeLessThanRowsToDeleteFromARegion");
HTable ht = createTable(tableName);
Table ht = createTable(tableName);
List<Put> puts = new ArrayList<Put>(100);
for (int j = 0; j < 100; j++) {
byte[] rowkey = Bytes.toBytes(j);
@ -123,7 +124,7 @@ public class TestBulkDeleteProtocol {
private long invokeBulkDeleteProtocol(byte[] tableName, final Scan scan, final int rowBatchSize,
final DeleteType deleteType, final Long timeStamp) throws Throwable {
HTable ht = new HTable(TEST_UTIL.getConfiguration(), tableName);
Table ht = new HTable(TEST_UTIL.getConfiguration(), tableName);
long noOfDeletedRows = 0L;
Batch.Call<BulkDeleteService, BulkDeleteResponse> callable =
new Batch.Call<BulkDeleteService, BulkDeleteResponse>() {
@ -155,7 +156,7 @@ public class TestBulkDeleteProtocol {
// @Ignore @Test
public void testBulkDeleteWithConditionBasedDelete() throws Throwable {
byte[] tableName = Bytes.toBytes("testBulkDeleteWithConditionBasedDelete");
HTable ht = createTable(tableName);
Table ht = createTable(tableName);
List<Put> puts = new ArrayList<Put>(100);
for (int j = 0; j < 100; j++) {
byte[] rowkey = Bytes.toBytes(j);
@ -185,7 +186,7 @@ public class TestBulkDeleteProtocol {
// @Ignore @Test
public void testBulkDeleteColumn() throws Throwable {
byte[] tableName = Bytes.toBytes("testBulkDeleteColumn");
HTable ht = createTable(tableName);
Table ht = createTable(tableName);
List<Put> puts = new ArrayList<Put>(100);
for (int j = 0; j < 100; j++) {
byte[] rowkey = Bytes.toBytes(j);
@ -218,7 +219,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);
HTable ht = new HTable(TEST_UTIL.getConfiguration(), tableName);
Table ht = new HTable(TEST_UTIL.getConfiguration(), tableName);
List<Put> puts = new ArrayList<Put>(100);
for (int j = 0; j < 100; j++) {
Put put = new Put(Bytes.toBytes(j));
@ -245,7 +246,7 @@ public class TestBulkDeleteProtocol {
// @Ignore @Test
public void testBulkDeleteColumnVersion() throws Throwable {
byte[] tableName = Bytes.toBytes("testBulkDeleteColumnVersion");
HTable ht = createTable(tableName);
Table ht = createTable(tableName);
List<Put> puts = new ArrayList<Put>(100);
for (int j = 0; j < 100; j++) {
Put put = new Put(Bytes.toBytes(j));
@ -293,7 +294,7 @@ public class TestBulkDeleteProtocol {
// @Ignore @Test
public void testBulkDeleteColumnVersionBasedOnTS() throws Throwable {
byte[] tableName = Bytes.toBytes("testBulkDeleteColumnVersionBasedOnTS");
HTable ht = createTable(tableName);
Table ht = createTable(tableName);
List<Put> puts = new ArrayList<Put>(100);
for (int j = 0; j < 100; j++) {
Put put = new Put(Bytes.toBytes(j));
@ -340,7 +341,7 @@ public class TestBulkDeleteProtocol {
// @Ignore @Test
public void testBulkDeleteWithNumberOfVersions() throws Throwable {
byte[] tableName = Bytes.toBytes("testBulkDeleteWithNumberOfVersions");
HTable ht = createTable(tableName);
Table ht = createTable(tableName);
List<Put> puts = new ArrayList<Put>(100);
for (int j = 0; j < 100; j++) {
Put put = new Put(Bytes.toBytes(j));
@ -422,13 +423,13 @@ public class TestBulkDeleteProtocol {
ht.close();
}
private HTable createTable(byte[] tableName) throws IOException {
private Table createTable(byte[] tableName) throws IOException {
HTableDescriptor htd = new HTableDescriptor(TableName.valueOf(tableName));
HColumnDescriptor hcd = new HColumnDescriptor(FAMILY1);
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);
HTable ht = new HTable(TEST_UTIL.getConfiguration(), tableName);
Table ht = new HTable(TEST_UTIL.getConfiguration(), tableName);
return ht;
}

View File

@ -23,16 +23,13 @@ import org.apache.hadoop.hbase.HBaseTestingUtility;
import org.apache.hadoop.hbase.MediumTests;
import org.apache.hadoop.hbase.client.HTable;
import org.apache.hadoop.hbase.client.Put;
import org.apache.hadoop.hbase.client.Table;
import org.apache.hadoop.hbase.client.coprocessor.Batch;
import org.apache.hadoop.hbase.coprocessor.CoprocessorHost;
import org.apache.hadoop.hbase.coprocessor.example.generated.ExampleProtos;
import org.apache.hadoop.hbase.ipc.BlockingRpcCallback;
import org.apache.hadoop.hbase.ipc.ServerRpcController;
import org.apache.hadoop.hbase.util.Bytes;
import org.junit.AfterClass;
import org.junit.BeforeClass;
import org.junit.Test;
import org.junit.Ignore;
import org.junit.experimental.categories.Category;
import java.io.IOException;
@ -72,7 +69,7 @@ public class TestRowCountEndpoint {
// @Ignore @Test
public void testEndpoint() throws Throwable {
HTable table = new HTable(CONF, TEST_TABLE);
Table table = new HTable(CONF, TEST_TABLE);
// insert some test rows
for (int i=0; i<5; i++) {

View File

@ -32,6 +32,7 @@ import org.apache.hadoop.hbase.client.Get;
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;
import org.apache.hadoop.hbase.coprocessor.CoprocessorHost;
import org.apache.hadoop.hbase.util.Bytes;
import org.apache.hadoop.hbase.util.EnvironmentEdgeManager;
@ -74,7 +75,7 @@ public class TestZooKeeperScanPolicyObserver {
.setTimeToLive(1);
desc.addFamily(hcd);
TEST_UTIL.getHBaseAdmin().createTable(desc);
HTable t = new HTable(new Configuration(TEST_UTIL.getConfiguration()), tableName);
Table t = new HTable(new Configuration(TEST_UTIL.getConfiguration()), tableName);
long now = EnvironmentEdgeManager.currentTime();
ZooKeeperWatcher zkw = new ZooKeeperWatcher(TEST_UTIL.getConfiguration(), "test", null);

View File

@ -25,6 +25,7 @@ import java.util.List;
import org.apache.hadoop.classification.InterfaceAudience;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.hbase.ClusterManager.ServiceType;
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;
@ -45,7 +46,7 @@ import com.google.common.collect.Sets;
@InterfaceAudience.Private
public class DistributedHBaseCluster extends HBaseCluster {
private HBaseAdmin admin;
private Admin admin;
private ClusterManager clusterManager;

View File

@ -31,6 +31,7 @@ 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.Scan;
import org.apache.hadoop.hbase.client.Table;
import org.apache.hadoop.hbase.filter.CompareFilter;
import org.apache.hadoop.hbase.filter.Filter;
import org.apache.hadoop.hbase.filter.SingleColumnValueFilter;
@ -222,7 +223,7 @@ 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;
HTable table = new HTable(conf, TABLE_NAME);
Table table = new HTable(conf, TABLE_NAME);
// Create multi-threaded writer and start it. We write multiple columns/CFs and verify
// their integrity, therefore multi-put is necessary.

View File

@ -48,6 +48,7 @@ import org.apache.hadoop.hbase.TableName;
import org.apache.hadoop.hbase.client.HTable;
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.util.Bytes;
import org.apache.hadoop.io.LongWritable;
import org.apache.hadoop.io.Text;
@ -153,7 +154,7 @@ public class IntegrationTestImportTsv implements Configurable, Tool {
assertEquals("Loading HFiles failed.",
0, ToolRunner.run(new LoadIncrementalHFiles(new Configuration(getConf())), args));
HTable table = null;
Table table = null;
Scan scan = new Scan() {{
setCacheBlocks(false);
setCaching(1000);

View File

@ -49,6 +49,7 @@ import org.apache.hadoop.hbase.chaos.actions.RestartActiveMasterAction;
import org.apache.hadoop.hbase.chaos.actions.RestartRsHoldingMetaAction;
import org.apache.hadoop.hbase.chaos.actions.RestartRsHoldingTableAction;
import org.apache.hadoop.hbase.chaos.factories.MonkeyConstants;
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.Put;
@ -56,6 +57,7 @@ import org.apache.hadoop.hbase.client.Result;
import org.apache.hadoop.hbase.client.ResultScanner;
import org.apache.hadoop.hbase.client.RetriesExhaustedException;
import org.apache.hadoop.hbase.client.Scan;
import org.apache.hadoop.hbase.client.Table;
import org.apache.hadoop.hbase.coprocessor.CoprocessorException;
import org.apache.hadoop.hbase.filter.KeyOnlyFilter;
import org.apache.hadoop.hbase.ipc.FatalConnectionException;
@ -461,7 +463,7 @@ public class IntegrationTestMTTR {
*/
static class PutCallable extends TimingCallable {
private final HTable table;
private final Table table;
public PutCallable(Future<?> f) throws IOException {
super(f);
@ -488,7 +490,7 @@ public class IntegrationTestMTTR {
* supplied future returns. Returns the max time taken to scan.
*/
static class ScanCallable extends TimingCallable {
private final HTable table;
private final Table table;
public ScanCallable(Future<?> f) throws IOException {
super(f);
@ -531,7 +533,7 @@ public class IntegrationTestMTTR {
@Override
protected boolean doAction() throws Exception {
HBaseAdmin admin = null;
Admin admin = null;
try {
admin = new HBaseAdmin(util.getConfiguration());
ClusterStatus status = admin.getClusterStatus();

View File

@ -50,6 +50,7 @@ import org.apache.hadoop.hbase.IntegrationTestingUtility;
import org.apache.hadoop.hbase.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.Get;
import org.apache.hadoop.hbase.client.HBaseAdmin;
import org.apache.hadoop.hbase.client.HConnection;
@ -60,6 +61,7 @@ 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.io.ImmutableBytesWritable;
import org.apache.hadoop.hbase.mapreduce.TableMapReduceUtil;
import org.apache.hadoop.hbase.mapreduce.TableMapper;
@ -448,7 +450,7 @@ public class IntegrationTestBigLinkedList extends IntegrationTestBase {
protected void createSchema() throws IOException {
Configuration conf = getConf();
HBaseAdmin admin = new HBaseAdmin(conf);
Admin admin = new HBaseAdmin(conf);
TableName tableName = getTableName(conf);
try {
if (!admin.tableExists(tableName)) {
@ -873,7 +875,7 @@ public class IntegrationTestBigLinkedList extends IntegrationTestBase {
System.exit(-1);
}
HTable table = new HTable(getConf(), getTableName(getConf()));
Table table = new HTable(getConf(), getTableName(getConf()));
Scan scan = new Scan();
scan.setBatch(10000);
@ -923,7 +925,7 @@ public class IntegrationTestBigLinkedList extends IntegrationTestBase {
org.apache.hadoop.hbase.client.Delete delete
= new org.apache.hadoop.hbase.client.Delete(val);
HTable table = new HTable(getConf(), getTableName(getConf()));
Table table = new HTable(getConf(), getTableName(getConf()));
table.delete(delete);
table.flushCommits();
@ -969,7 +971,7 @@ 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;
HTable table = new HTable(getConf(), getTableName(getConf()));
Table table = new HTable(getConf(), 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
@ -1005,7 +1007,7 @@ public class IntegrationTestBigLinkedList extends IntegrationTestBase {
return 0;
}
private static CINode findStartNode(HTable table, byte[] startKey) throws IOException {
private static CINode findStartNode(Table table, byte[] startKey) throws IOException {
Scan scan = new Scan();
scan.setStartRow(startKey);
scan.setBatch(1);
@ -1028,7 +1030,7 @@ public class IntegrationTestBigLinkedList extends IntegrationTestBase {
return null;
}
private CINode getNode(byte[] row, HTable table, CINode node) throws IOException {
private CINode getNode(byte[] row, Table table, CINode node) throws IOException {
Get get = new Get(row);
get.addColumn(FAMILY_NAME, COLUMN_PREV);
Result result = table.get(get);

View File

@ -38,6 +38,7 @@ import org.apache.hadoop.hbase.IntegrationTestingUtility;
import org.apache.hadoop.hbase.IntegrationTests;
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.Delete;
import org.apache.hadoop.hbase.client.HBaseAdmin;
import org.apache.hadoop.hbase.client.HConnection;
@ -46,6 +47,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.io.hfile.HFile;
import org.apache.hadoop.hbase.mapreduce.Import;
@ -130,7 +132,7 @@ public class IntegrationTestBigLinkedListWithVisibility extends IntegrationTestB
if(!acl) {
LOG.info("No ACL available.");
}
HBaseAdmin admin = new HBaseAdmin(getConf());
Admin admin = new HBaseAdmin(getConf());
for (int i = 0; i < DEFAULT_TABLES_COUNT; i++) {
TableName tableName = IntegrationTestBigLinkedListWithVisibility.getTableName(i);
createTable(admin, tableName, false, acl);
@ -140,7 +142,7 @@ public class IntegrationTestBigLinkedListWithVisibility extends IntegrationTestB
admin.close();
}
private void createTable(HBaseAdmin admin, TableName tableName, boolean setVersion,
private void createTable(Admin admin, TableName tableName, boolean setVersion,
boolean acl) throws IOException {
if (!admin.tableExists(tableName)) {
HTableDescriptor htd = new HTableDescriptor(tableName);
@ -170,8 +172,8 @@ public class IntegrationTestBigLinkedListWithVisibility extends IntegrationTestB
}
static class VisibilityGeneratorMapper extends GeneratorMapper {
HTable[] tables = new HTable[DEFAULT_TABLES_COUNT];
HTable commonTable = null;
Table[] tables = new Table[DEFAULT_TABLES_COUNT];
Table commonTable = null;
@Override
protected void setup(org.apache.hadoop.mapreduce.Mapper.Context context) throws IOException,

View File

@ -448,7 +448,7 @@ public void cleanUpCluster() throws Exception {
HTableDescriptor htd = new HTableDescriptor(table);
htd.addFamily(new HColumnDescriptor(TEST_FAMILY));
HBaseAdmin admin = new HBaseAdmin(getConf());
Admin admin = new HBaseAdmin(getConf());
if (doLoad) {
admin.createTable(htd, Bytes.toBytes(0L), Bytes.toBytes(-1L), numPresplits);
doLoad(getConf(), htd);

View File

@ -39,8 +39,8 @@ import org.apache.hadoop.hbase.chaos.factories.MonkeyFactory;
import org.apache.hadoop.hbase.client.Admin;
import org.apache.hadoop.hbase.client.Consistency;
import org.apache.hadoop.hbase.client.Get;
import org.apache.hadoop.hbase.client.HTableInterface;
import org.apache.hadoop.hbase.client.Result;
import org.apache.hadoop.hbase.client.Table;
import org.apache.hadoop.hbase.regionserver.StorefileRefresherChore;
import org.apache.hadoop.hbase.util.LoadTestTool;
import org.apache.hadoop.hbase.util.MultiThreadedReader;
@ -326,7 +326,7 @@ public class IntegrationTestTimeBoundedRequestsWithRegionReplicas extends Integr
@Override
protected void verifyResultsAndUpdateMetrics(boolean verify, Get[] gets, long elapsedNano,
Result[] results, HTableInterface table, boolean isNullExpected)
Result[] results, Table table, boolean isNullExpected)
throws IOException {
super.verifyResultsAndUpdateMetrics(verify, gets, elapsedNano, results, table, isNullExpected);
for (Result r : results) {

View File

@ -33,6 +33,7 @@ import org.apache.hadoop.hbase.HConstants;
import org.apache.hadoop.hbase.HTableDescriptor;
import org.apache.hadoop.hbase.IntegrationTestingUtility;
import org.apache.hadoop.hbase.IntegrationTests;
import org.apache.hadoop.hbase.client.Admin;
import org.apache.hadoop.hbase.client.HBaseAdmin;
import org.apache.hadoop.hbase.client.Put;
import org.apache.hadoop.hbase.client.Result;
@ -369,7 +370,7 @@ public class IntegrationTestWithCellVisibilityLoadAndVerify extends IntegrationT
HTableDescriptor htd = new HTableDescriptor(getTablename());
htd.addFamily(new HColumnDescriptor(TEST_FAMILY));
HBaseAdmin admin = new HBaseAdmin(getConf());
Admin admin = new HBaseAdmin(getConf());
try {
admin.createTable(htd, Bytes.toBytes(0L), Bytes.toBytes(-1L), numPresplits);
} finally {

View File

@ -31,6 +31,7 @@ import org.apache.hadoop.hbase.client.Put;
import org.apache.hadoop.hbase.client.Result;
import org.apache.hadoop.hbase.client.ResultScanner;
import org.apache.hadoop.hbase.client.Scan;
import org.apache.hadoop.hbase.client.Table;
import org.apache.hadoop.hbase.util.AbstractHBaseTool;
import org.apache.hadoop.hbase.util.Bytes;
import org.apache.hadoop.util.ToolRunner;
@ -123,7 +124,7 @@ public class IntegrationTestSendTraceRequests extends AbstractHBaseTool {
ResultScanner rs = null;
try {
innerScope = Trace.startSpan("Scan", Sampler.ALWAYS);
HTable ht = new HTable(util.getConfiguration(), tableName);
Table ht = new HTable(util.getConfiguration(), tableName);
Scan s = new Scan();
s.setStartRow(Bytes.toBytes(rowKeyQueue.take()));
s.setBatch(7);
@ -171,7 +172,7 @@ public class IntegrationTestSendTraceRequests extends AbstractHBaseTool {
public void run() {
HTable ht = null;
Table ht = null;
try {
ht = new HTable(util.getConfiguration(), tableName);
} catch (IOException e) {

View File

@ -29,6 +29,7 @@ import org.apache.commons.logging.LogFactory;
import org.apache.hadoop.classification.InterfaceAudience;
import org.apache.hadoop.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.regionserver.HRegionServer;
import org.apache.hadoop.hbase.security.User;
@ -462,7 +463,7 @@ public class LocalHBaseCluster {
Configuration conf = HBaseConfiguration.create();
LocalHBaseCluster cluster = new LocalHBaseCluster(conf);
cluster.startup();
HBaseAdmin admin = new HBaseAdmin(conf);
Admin admin = new HBaseAdmin(conf);
HTableDescriptor htd =
new HTableDescriptor(TableName.valueOf(cluster.getClass().getName()));
admin.createTable(htd);

View File

@ -47,6 +47,7 @@ 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.client.Table;
import org.apache.hadoop.hbase.util.Bytes;
import org.apache.hadoop.hbase.util.CoprocessorClassLoader;
import org.apache.hadoop.hbase.util.SortedCopyOnWriteSet;

View File

@ -26,6 +26,7 @@ import org.apache.hadoop.classification.InterfaceAudience;
import org.apache.hadoop.classification.InterfaceStability;
import org.apache.hadoop.hbase.HBaseConfiguration;
import org.apache.hadoop.hbase.client.HTable;
import org.apache.hadoop.hbase.client.RegionLocator;
import org.apache.hadoop.hbase.io.ImmutableBytesWritable;
import org.apache.hadoop.hbase.util.Bytes;
import org.apache.hadoop.mapred.JobConf;
@ -46,7 +47,7 @@ import org.apache.hadoop.mapred.Partitioner;
public class HRegionPartitioner<K2,V2>
implements Partitioner<ImmutableBytesWritable, V2> {
private static final Log LOG = LogFactory.getLog(HRegionPartitioner.class);
private HTable table;
private RegionLocator table;
private byte[][] startKeys;
public void configure(JobConf job) {

View File

@ -28,6 +28,7 @@ import org.apache.hadoop.fs.FileSystem;
import org.apache.hadoop.hbase.HBaseConfiguration;
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.io.ImmutableBytesWritable;
import org.apache.hadoop.fs.FileAlreadyExistsException;
import org.apache.hadoop.mapred.InvalidJobConfException;
@ -56,14 +57,14 @@ FileOutputFormat<ImmutableBytesWritable, Put> {
*/
protected static class TableRecordWriter
implements RecordWriter<ImmutableBytesWritable, Put> {
private HTable m_table;
private Table m_table;
/**
* Instantiate a TableRecordWriter with the HBase HClient for writing.
*
* @param table
*/
public TableRecordWriter(HTable table) {
public TableRecordWriter(Table table) {
m_table = table;
}

View File

@ -24,6 +24,7 @@ import org.apache.hadoop.classification.InterfaceAudience;
import org.apache.hadoop.classification.InterfaceStability;
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.RecordReader;
@ -62,7 +63,7 @@ implements RecordReader<ImmutableBytesWritable, Result> {
/**
* @param htable the {@link HTable} to scan.
*/
public void setHTable(HTable htable) {
public void setHTable(Table htable) {
this.recordReaderImpl.setHTable(htable);
}

View File

@ -30,6 +30,7 @@ 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.filter.Filter;
import org.apache.hadoop.hbase.io.ImmutableBytesWritable;
import org.apache.hadoop.hbase.mapreduce.TableInputFormat;
@ -52,7 +53,7 @@ public class TableRecordReaderImpl {
private byte [] lastSuccessfulRow;
private Filter trrRowFilter;
private ResultScanner scanner;
private HTable htable;
private Table htable;
private byte [][] trrInputColumns;
private long timestamp;
private int rowcount;
@ -116,7 +117,7 @@ public class TableRecordReaderImpl {
/**
* @param htable the {@link HTable} to scan.
*/
public void setHTable(HTable htable) {
public void setHTable(Table htable) {
Configuration conf = htable.getConfiguration();
logScannerActivity = conf.getBoolean(
ScannerCallable.LOG_SCANNER_ACTIVITY, false);

View File

@ -36,6 +36,7 @@ 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.Scan;
import org.apache.hadoop.hbase.client.Table;
import org.apache.hadoop.hbase.security.visibility.Authorizations;
import org.apache.hadoop.hbase.security.visibility.VisibilityLabelOrdinalProvider;
import org.apache.hadoop.hbase.security.visibility.VisibilityUtils;
@ -66,7 +67,7 @@ public class DefaultVisibilityExpressionResolver implements VisibilityExpression
public void init() {
// Reading all the labels and ordinal.
// This scan should be done by user with global_admin previliges.. Ensure that it works
HTable labelsTable = null;
Table labelsTable = null;
try {
labelsTable = new HTable(conf, LABELS_TABLE_NAME);
} catch (TableNotFoundException e) {

View File

@ -29,6 +29,7 @@ import org.apache.hadoop.classification.InterfaceStability;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.hbase.KeyValue;
import org.apache.hadoop.hbase.client.HTable;
import org.apache.hadoop.hbase.client.Table;
import org.apache.hadoop.hbase.io.ImmutableBytesWritable;
import org.apache.hadoop.hbase.io.compress.Compression.Algorithm;
import org.apache.hadoop.hbase.io.encoding.DataBlockEncoding;
@ -160,7 +161,7 @@ public class HFileOutputFormat extends FileOutputFormat<ImmutableBytesWritable,
@edu.umd.cs.findbugs.annotations.SuppressWarnings(
value="RCN_REDUNDANT_NULLCHECK_OF_NONNULL_VALUE")
@VisibleForTesting
static void configureCompression(HTable table, Configuration conf) throws IOException {
static void configureCompression(Table table, Configuration conf) throws IOException {
HFileOutputFormat2.configureCompression(table, conf);
}
@ -174,7 +175,7 @@ public class HFileOutputFormat extends FileOutputFormat<ImmutableBytesWritable,
* on failure to read column family descriptors
*/
@VisibleForTesting
static void configureBlockSize(HTable table, Configuration conf) throws IOException {
static void configureBlockSize(Table table, Configuration conf) throws IOException {
HFileOutputFormat2.configureBlockSize(table, conf);
}
@ -188,7 +189,7 @@ public class HFileOutputFormat extends FileOutputFormat<ImmutableBytesWritable,
* on failure to read column family descriptors
*/
@VisibleForTesting
static void configureBloomType(HTable table, Configuration conf) throws IOException {
static void configureBloomType(Table table, Configuration conf) throws IOException {
HFileOutputFormat2.configureBloomType(table, conf);
}
@ -202,7 +203,7 @@ public class HFileOutputFormat extends FileOutputFormat<ImmutableBytesWritable,
* on failure to read column family descriptors
*/
@VisibleForTesting
static void configureDataBlockEncoding(HTable table,
static void configureDataBlockEncoding(Table table,
Configuration conf) throws IOException {
HFileOutputFormat2.configureDataBlockEncoding(table, conf);
}

View File

@ -45,6 +45,8 @@ import org.apache.hadoop.hbase.KeyValue;
import org.apache.hadoop.hbase.KeyValueUtil;
import org.apache.hadoop.hbase.client.HTable;
import org.apache.hadoop.hbase.client.Put;
import org.apache.hadoop.hbase.client.RegionLocator;
import org.apache.hadoop.hbase.client.Table;
import org.apache.hadoop.hbase.io.ImmutableBytesWritable;
import org.apache.hadoop.hbase.io.compress.Compression;
import org.apache.hadoop.hbase.io.compress.Compression.Algorithm;
@ -284,7 +286,7 @@ public class HFileOutputFormat2
* Return the start keys of all of the regions in this table,
* as a list of ImmutableBytesWritable.
*/
private static List<ImmutableBytesWritable> getRegionStartKeys(HTable table)
private static List<ImmutableBytesWritable> getRegionStartKeys(RegionLocator table)
throws IOException {
byte[][] byteKeys = table.getStartKeys();
ArrayList<ImmutableBytesWritable> ret =
@ -544,7 +546,7 @@ public class HFileOutputFormat2
value="RCN_REDUNDANT_NULLCHECK_OF_NONNULL_VALUE")
@VisibleForTesting
static void configureCompression(
HTable table, Configuration conf) throws IOException {
Table table, Configuration conf) throws IOException {
StringBuilder compressionConfigValue = new StringBuilder();
HTableDescriptor tableDescriptor = table.getTableDescriptor();
if(tableDescriptor == null){
@ -578,7 +580,7 @@ public class HFileOutputFormat2
*/
@VisibleForTesting
static void configureBlockSize(
HTable table, Configuration conf) throws IOException {
Table table, Configuration conf) throws IOException {
StringBuilder blockSizeConfigValue = new StringBuilder();
HTableDescriptor tableDescriptor = table.getTableDescriptor();
if (tableDescriptor == null) {
@ -612,7 +614,7 @@ public class HFileOutputFormat2
*/
@VisibleForTesting
static void configureBloomType(
HTable table, Configuration conf) throws IOException {
Table table, Configuration conf) throws IOException {
HTableDescriptor tableDescriptor = table.getTableDescriptor();
if (tableDescriptor == null) {
// could happen with mock table instance
@ -647,7 +649,7 @@ public class HFileOutputFormat2
* on failure to read column family descriptors
*/
@VisibleForTesting
static void configureDataBlockEncoding(HTable table,
static void configureDataBlockEncoding(Table table,
Configuration conf) throws IOException {
HTableDescriptor tableDescriptor = table.getTableDescriptor();
if (tableDescriptor == null) {

View File

@ -28,6 +28,7 @@ import org.apache.hadoop.conf.Configurable;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.hbase.HBaseConfiguration;
import org.apache.hadoop.hbase.client.HTable;
import org.apache.hadoop.hbase.client.RegionLocator;
import org.apache.hadoop.hbase.io.ImmutableBytesWritable;
import org.apache.hadoop.hbase.util.Bytes;
import org.apache.hadoop.mapreduce.Partitioner;
@ -53,7 +54,7 @@ implements Configurable {
private static final Log LOG = LogFactory.getLog(HRegionPartitioner.class);
private Configuration conf = null;
private HTable table;
private RegionLocator table;
private byte[][] startKeys;
/**

View File

@ -39,6 +39,7 @@ import org.apache.hadoop.hbase.HColumnDescriptor;
import org.apache.hadoop.hbase.HConstants;
import org.apache.hadoop.hbase.HTableDescriptor;
import org.apache.hadoop.hbase.TableName;
import org.apache.hadoop.hbase.client.Admin;
import org.apache.hadoop.hbase.client.HBaseAdmin;
import org.apache.hadoop.hbase.client.HTable;
import org.apache.hadoop.hbase.client.Put;
@ -469,7 +470,7 @@ public class ImportTsv extends Configured implements Tool {
return job;
}
private static void createTable(HBaseAdmin admin, String tableName, String[] columns)
private static void createTable(Admin admin, String tableName, String[] columns)
throws IOException {
HTableDescriptor htd = new HTableDescriptor(TableName.valueOf(tableName));
Set<String> cfSet = new HashSet<String>();

View File

@ -41,7 +41,6 @@ import java.util.concurrent.Future;
import java.util.concurrent.LinkedBlockingQueue;
import java.util.concurrent.ThreadPoolExecutor;
import java.util.concurrent.TimeUnit;
import java.util.concurrent.atomic.AtomicLong;
import org.apache.commons.lang.mutable.MutableInt;
import org.apache.commons.logging.Log;
@ -63,11 +62,13 @@ import org.apache.hadoop.hbase.KeyValue;
import org.apache.hadoop.hbase.KeyValueUtil;
import org.apache.hadoop.hbase.TableName;
import org.apache.hadoop.hbase.TableNotFoundException;
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.HTable;
import org.apache.hadoop.hbase.client.RegionServerCallable;
import org.apache.hadoop.hbase.client.RpcRetryingCallerFactory;
import org.apache.hadoop.hbase.client.Table;
import org.apache.hadoop.hbase.client.coprocessor.SecureBulkLoadClient;
import org.apache.hadoop.hbase.io.HFileLink;
import org.apache.hadoop.hbase.io.HalfStoreFileReader;
@ -106,7 +107,7 @@ import java.util.UUID;
@InterfaceStability.Stable
public class LoadIncrementalHFiles extends Configured implements Tool {
private static final Log LOG = LogFactory.getLog(LoadIncrementalHFiles.class);
private HBaseAdmin hbAdmin;
private Admin hbAdmin;
public static final String NAME = "completebulkload";
public static final String MAX_FILES_PER_REGION_PER_FAMILY
@ -359,7 +360,7 @@ public class LoadIncrementalHFiles extends Configured implements Tool {
* them. Any failures are re-queued for another pass with the
* groupOrSplitPhase.
*/
protected void bulkLoadPhase(final HTable table, final HConnection conn,
protected void bulkLoadPhase(final Table table, final HConnection conn,
ExecutorService pool, Deque<LoadQueueItem> queue,
final Multimap<ByteBuffer, LoadQueueItem> regionGroups) throws IOException {
// atomically bulk load the groups.
@ -482,7 +483,7 @@ public class LoadIncrementalHFiles extends Configured implements Tool {
}
protected List<LoadQueueItem> splitStoreFile(final LoadQueueItem item,
final HTable table, byte[] startKey,
final Table table, byte[] startKey,
byte[] splitKey) throws IOException {
final Path hfilePath = item.hfilePath;
@ -646,7 +647,7 @@ public class LoadIncrementalHFiles extends Configured implements Tool {
if(!userProvider.isHBaseSecurityEnabled()) {
success = ProtobufUtil.bulkLoadHFile(getStub(), famPaths, regionName, assignSeqIds);
} else {
HTable table = new HTable(conn.getConfiguration(), getTableName());
Table table = new HTable(conn.getConfiguration(), getTableName());
secureClient = new SecureBulkLoadClient(table);
success = secureClient.bulkLoadHFiles(famPaths, fsDelegationToken.getUserToken(),
bulkToken, getLocation().getRegionInfo().getStartKey());

View File

@ -31,6 +31,7 @@ import org.apache.hadoop.hbase.HRegionLocation;
import org.apache.hadoop.hbase.client.HTable;
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.util.Bytes;
import org.apache.hadoop.hbase.util.Pair;
@ -84,7 +85,7 @@ public abstract class MultiTableInputFormatBase extends
+ " previous error. Please look at the previous logs lines from"
+ " the task's full log for more details.");
}
HTable table =
Table table =
new HTable(context.getConfiguration(), tSplit.getTableName());
TableRecordReader trr = this.tableRecordReader;

View File

@ -19,10 +19,8 @@
package org.apache.hadoop.hbase.mapreduce;
import java.io.IOException;
import java.io.InterruptedIOException;
import java.net.InetAddress;
import java.net.InetSocketAddress;
import java.text.MessageFormat;
import java.util.ArrayList;
import java.util.HashMap;
import java.util.List;

View File

@ -32,6 +32,7 @@ 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.Table;
import org.apache.hadoop.hbase.zookeeper.ZKUtil;
import org.apache.hadoop.mapreduce.JobContext;
import org.apache.hadoop.mapreduce.OutputCommitter;
@ -88,14 +89,14 @@ implements Configurable {
extends RecordWriter<KEY, Mutation> {
/** The table to write to. */
private HTable table;
private Table table;
/**
* Instantiate a TableRecordWriter with the HBase HClient for writing.
*
* @param table The table to write to.
*/
public TableRecordWriter(HTable table) {
public TableRecordWriter(Table table) {
this.table = table;
}

View File

@ -25,6 +25,7 @@ import org.apache.hadoop.classification.InterfaceStability;
import org.apache.hadoop.hbase.client.HTable;
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.mapreduce.InputSplit;
import org.apache.hadoop.mapreduce.RecordReader;
@ -57,7 +58,7 @@ extends RecordReader<ImmutableBytesWritable, Result> {
*
* @param htable The {@link HTable} to scan.
*/
public void setHTable(HTable htable) {
public void setHTable(Table htable) {
this.recordReaderImpl.setHTable(htable);
}

View File

@ -31,6 +31,7 @@ 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.client.metrics.ScanMetrics;
import org.apache.hadoop.hbase.io.ImmutableBytesWritable;
import org.apache.hadoop.hbase.protobuf.ProtobufUtil;
@ -58,7 +59,7 @@ public class TableRecordReaderImpl {
private ResultScanner scanner = null;
private Scan scan = null;
private Scan currentScan = null;
private HTable htable = null;
private Table htable = null;
private byte[] lastSuccessfulRow = null;
private ImmutableBytesWritable key = null;
private Result value = null;
@ -121,7 +122,7 @@ public class TableRecordReaderImpl {
*
* @param htable The {@link HTable} to scan.
*/
public void setHTable(HTable htable) {
public void setHTable(Table htable) {
Configuration conf = htable.getConfiguration();
logScannerActivity = conf.getBoolean(
ScannerCallable.LOG_SCANNER_ACTIVITY, false);

View File

@ -33,6 +33,7 @@ import org.apache.hadoop.hbase.client.Put;
import org.apache.hadoop.hbase.client.Result;
import org.apache.hadoop.hbase.client.ResultScanner;
import org.apache.hadoop.hbase.client.Scan;
import org.apache.hadoop.hbase.client.Table;
import org.apache.hadoop.hbase.io.ImmutableBytesWritable;
import org.apache.hadoop.hbase.mapreduce.TableInputFormat;
import org.apache.hadoop.hbase.mapreduce.TableMapReduceUtil;
@ -120,7 +121,7 @@ public class VerifyReplication extends Configured implements Tool {
Configuration peerConf = HBaseConfiguration.create(conf);
ZKUtil.applyClusterKeyToConf(peerConf, zkClusterKey);
HTable replicatedTable = new HTable(peerConf, conf.get(NAME + ".tableName"));
Table replicatedTable = new HTable(peerConf, conf.get(NAME + ".tableName"));
scan.setStartRow(value.getRow());
replicatedScanner = replicatedTable.getScanner(scan);
return null;

View File

@ -30,7 +30,6 @@ import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
import org.apache.hadoop.classification.InterfaceAudience;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.fs.FileSystem;
import org.apache.hadoop.hbase.CoordinatedStateManager;
import org.apache.hadoop.hbase.CoordinatedStateManagerFactory;
import org.apache.hadoop.hbase.MasterNotRunningException;
@ -38,6 +37,7 @@ import org.apache.hadoop.hbase.ZNodeClearer;
import org.apache.hadoop.hbase.HConstants;
import org.apache.hadoop.hbase.LocalHBaseCluster;
import org.apache.hadoop.hbase.ZooKeeperConnectionException;
import org.apache.hadoop.hbase.client.Admin;
import org.apache.hadoop.hbase.client.HBaseAdmin;
import org.apache.hadoop.hbase.regionserver.HRegionServer;
import org.apache.hadoop.hbase.util.JVMClusterUtil;
@ -209,7 +209,7 @@ public class HMasterCommandLine extends ServerCommandLine {
}
private int stopMaster() {
HBaseAdmin adm = null;
Admin adm = null;
try {
Configuration conf = getConf();
// Don't try more than once

View File

@ -47,6 +47,7 @@ import org.apache.hadoop.hbase.HConstants;
import org.apache.hadoop.hbase.HRegionInfo;
import org.apache.hadoop.hbase.ServerName;
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.HConnection;
import org.apache.hadoop.hbase.client.HConnectionManager;
@ -92,7 +93,7 @@ public class RegionPlacementMaintainer {
private Configuration conf;
private final boolean enforceLocality;
private final boolean enforceMinAssignmentMove;
private HBaseAdmin admin;
private Admin admin;
private RackManager rackManager;
private Set<TableName> targetTableSet;
@ -127,7 +128,7 @@ public class RegionPlacementMaintainer {
* @return the cached HBaseAdmin
* @throws IOException
*/
private HBaseAdmin getHBaseAdmin() throws IOException {
private Admin getHBaseAdmin() throws IOException {
if (this.admin == null) {
this.admin = new HBaseAdmin(this.conf);
}

View File

@ -45,6 +45,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.ResultScanner;
import org.apache.hadoop.hbase.client.Table;
import org.apache.hadoop.hbase.constraint.ConstraintException;
import org.apache.hadoop.hbase.master.handler.CreateTableHandler;
import org.apache.hadoop.hbase.protobuf.ProtobufUtil;
@ -107,7 +108,7 @@ public class TableNamespaceManager {
isTableAvailableAndInitialized();
}
private synchronized HTable getNamespaceTable() throws IOException {
private synchronized Table getNamespaceTable() throws IOException {
if (!isTableAvailableAndInitialized()) {
throw new IOException(this.getClass().getName() + " isn't ready to serve");
}
@ -125,14 +126,14 @@ public class TableNamespaceManager {
}
public synchronized void update(NamespaceDescriptor ns) throws IOException {
HTable table = getNamespaceTable();
Table table = getNamespaceTable();
if (get(table, ns.getName()) == null) {
throw new NamespaceNotFoundException(ns.getName());
}
upsert(table, ns);
}
private NamespaceDescriptor get(HTable table, String name) throws IOException {
private NamespaceDescriptor get(Table table, String name) throws IOException {
Result res = table.get(new Get(Bytes.toBytes(name)));
if (res.isEmpty()) {
return null;
@ -144,7 +145,7 @@ public class TableNamespaceManager {
HBaseProtos.NamespaceDescriptor.parseFrom(val));
}
private void create(HTable table, NamespaceDescriptor ns) throws IOException {
private void create(Table table, NamespaceDescriptor ns) throws IOException {
if (get(table, ns.getName()) != null) {
throw new NamespaceExistException(ns.getName());
}
@ -154,7 +155,7 @@ public class TableNamespaceManager {
upsert(table, ns);
}
private void upsert(HTable table, NamespaceDescriptor ns) throws IOException {
private void upsert(Table table, NamespaceDescriptor ns) throws IOException {
Put p = new Put(Bytes.toBytes(ns.getName()));
p.addImmutable(HTableDescriptor.NAMESPACE_FAMILY_INFO_BYTES,
HTableDescriptor.NAMESPACE_COL_DESC_BYTES,

View File

@ -41,6 +41,7 @@ import org.apache.hadoop.hbase.MetaTableAccessor;
import org.apache.hadoop.hbase.client.HConnection;
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.master.RackManager;
import org.apache.hadoop.hbase.protobuf.ProtobufUtil;
import org.apache.hadoop.hbase.protobuf.generated.HBaseProtos;
@ -120,7 +121,7 @@ public class FavoredNodeAssignmentHelper {
}
}
// Write the region assignments to the meta table.
HTable metaTable = null;
Table metaTable = null;
try {
metaTable = new HTable(conf, TableName.META_TABLE_NAME);
metaTable.put(puts);

View File

@ -36,6 +36,7 @@ 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.Scan;
import org.apache.hadoop.hbase.client.Table;
import org.apache.hadoop.hbase.executor.EventType;
import org.apache.hadoop.hbase.master.HMaster;
import org.apache.hadoop.hbase.master.MasterCoprocessorHost;
@ -94,7 +95,7 @@ public class ModifyTableHandler extends TableEventHandler {
Set<byte[]> tableRows = new HashSet<byte[]>();
Scan scan = MetaTableAccessor.getScanForTableName(table);
scan.addColumn(HConstants.CATALOG_FAMILY, HConstants.REGIONINFO_QUALIFIER);
HTable htable = null;
Table htable = null;
try {
htable = new HTable(masterServices.getConfiguration(), TableName.META_TABLE_NAME);
ResultScanner resScanner = htable.getScanner(scan);

View File

@ -46,10 +46,10 @@ import org.apache.hadoop.hbase.Stoppable;
import org.apache.hadoop.hbase.client.Delete;
import org.apache.hadoop.hbase.client.HConnection;
import org.apache.hadoop.hbase.client.HConnectionManager;
import org.apache.hadoop.hbase.client.HTableInterface;
import org.apache.hadoop.hbase.client.Mutation;
import org.apache.hadoop.hbase.client.Put;
import org.apache.hadoop.hbase.client.Row;
import org.apache.hadoop.hbase.client.Table;
import org.apache.hadoop.hbase.protobuf.generated.AdminProtos.WALEntry;
import org.apache.hadoop.hbase.protobuf.generated.HBaseProtos;
@ -230,7 +230,7 @@ public class ReplicationSink {
if (allRows.isEmpty()) {
return;
}
HTableInterface table = null;
Table table = null;
try {
table = this.sharedHtableCon.getTable(tableName);
for (List<Row> rows : allRows) {

View File

@ -23,7 +23,7 @@ import java.io.IOException;
import org.apache.hadoop.classification.InterfaceAudience;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.hbase.client.HBaseAdmin;
import org.apache.hadoop.hbase.client.HTableInterface;
import org.apache.hadoop.hbase.client.Table;
import org.apache.hadoop.hbase.filter.ParseFilter;
import org.apache.hadoop.hbase.security.UserProvider;
import org.apache.hadoop.hbase.util.ConnectionCache;
@ -100,7 +100,7 @@ public class RESTServlet implements Constants {
/**
* Caller closes the table afterwards.
*/
HTableInterface getTable(String tableName) throws IOException {
Table getTable(String tableName) throws IOException {
return connectionCache.getTable(tableName);
}

View File

@ -45,8 +45,8 @@ import org.apache.hadoop.hbase.CellUtil;
import org.apache.hadoop.hbase.HConstants;
import org.apache.hadoop.hbase.KeyValue;
import org.apache.hadoop.hbase.client.Delete;
import org.apache.hadoop.hbase.client.HTableInterface;
import org.apache.hadoop.hbase.client.Put;
import org.apache.hadoop.hbase.client.Table;
import org.apache.hadoop.hbase.rest.model.CellModel;
import org.apache.hadoop.hbase.rest.model.CellSetModel;
import org.apache.hadoop.hbase.rest.model.RowModel;
@ -182,7 +182,7 @@ public class RowResource extends ResourceBase {
.build();
}
HTableInterface table = null;
Table table = null;
try {
List<RowModel> rows = model.getRows();
List<Put> puts = new ArrayList<Put>();
@ -253,7 +253,7 @@ public class RowResource extends ResourceBase {
.type(MIMETYPE_TEXT).entity("Forbidden" + CRLF)
.build();
}
HTableInterface table = null;
Table table = null;
try {
byte[] row = rowspec.getRow();
byte[][] columns = rowspec.getColumns();
@ -393,7 +393,7 @@ public class RowResource extends ResourceBase {
}
}
}
HTableInterface table = null;
Table table = null;
try {
table = servlet.getTable(tableResource.getName());
table.delete(delete);
@ -422,7 +422,7 @@ public class RowResource extends ResourceBase {
* @return Response 200 OK, 304 Not modified, 400 Bad request
*/
Response checkAndPut(final CellSetModel model) {
HTableInterface table = null;
Table table = null;
try {
table = servlet.getTable(tableResource.getName());
if (model.getRows().size() != 1) {
@ -513,7 +513,7 @@ public class RowResource extends ResourceBase {
* @return Response 200 OK, 304 Not modified, 400 Bad request
*/
Response checkAndDelete(final CellSetModel model) {
HTableInterface table = null;
Table table = null;
Delete delete = null;
try {
table = servlet.getTable(tableResource.getName());

View File

@ -30,8 +30,8 @@ import org.apache.hadoop.hbase.Cell;
import org.apache.hadoop.hbase.DoNotRetryIOException;
import org.apache.hadoop.hbase.KeyValue;
import org.apache.hadoop.hbase.client.Get;
import org.apache.hadoop.hbase.client.HTableInterface;
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.util.StringUtils;
@ -45,7 +45,7 @@ public class RowResultGenerator extends ResultGenerator {
public RowResultGenerator(final String tableName, final RowSpec rowspec,
final Filter filter, final boolean cacheBlocks)
throws IllegalArgumentException, IOException {
HTableInterface table = RESTServlet.getInstance().getTable(tableName);
Table table = RESTServlet.getInstance().getTable(tableName);
try {
Get get = new Get(rowspec.getRow());
if (rowspec.hasColumns()) {

View File

@ -28,10 +28,10 @@ import org.apache.hadoop.classification.InterfaceAudience;
import org.apache.hadoop.hbase.Cell;
import org.apache.hadoop.hbase.KeyValue;
import org.apache.hadoop.hbase.UnknownScannerException;
import org.apache.hadoop.hbase.client.HTableInterface;
import org.apache.hadoop.hbase.client.Result;
import org.apache.hadoop.hbase.client.ResultScanner;
import org.apache.hadoop.hbase.client.Scan;
import org.apache.hadoop.hbase.client.Table;
import org.apache.hadoop.hbase.filter.Filter;
import org.apache.hadoop.hbase.rest.model.ScannerModel;
import org.apache.hadoop.hbase.security.visibility.Authorizations;
@ -67,7 +67,7 @@ public class ScannerResultGenerator extends ResultGenerator {
public ScannerResultGenerator(final String tableName, final RowSpec rowspec,
final Filter filter, final int caching, final boolean cacheBlocks)
throws IllegalArgumentException, IOException {
HTableInterface table = RESTServlet.getInstance().getTable(tableName);
Table table = RESTServlet.getInstance().getTable(tableName);
try {
Scan scan;
if (rowspec.hasEndRow()) {

View File

@ -44,7 +44,7 @@ import org.apache.hadoop.hbase.TableExistsException;
import org.apache.hadoop.hbase.TableName;
import org.apache.hadoop.hbase.TableNotFoundException;
import org.apache.hadoop.hbase.client.HBaseAdmin;
import org.apache.hadoop.hbase.client.HTableInterface;
import org.apache.hadoop.hbase.client.Table;
import org.apache.hadoop.hbase.rest.model.ColumnSchemaModel;
import org.apache.hadoop.hbase.rest.model.TableSchemaModel;
import org.apache.hadoop.hbase.util.Bytes;
@ -74,7 +74,7 @@ public class SchemaResource extends ResourceBase {
private HTableDescriptor getTableSchema() throws IOException,
TableNotFoundException {
HTableInterface table = servlet.getTable(tableResource.getName());
Table table = servlet.getTable(tableResource.getName());
try {
return table.getTableDescriptor();
} finally {

View File

@ -35,8 +35,8 @@ import org.apache.commons.lang.StringUtils;
import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
import org.apache.hadoop.classification.InterfaceAudience;
import org.apache.hadoop.hbase.client.HTableInterface;
import org.apache.hadoop.hbase.client.Scan;
import org.apache.hadoop.hbase.client.Table;
import org.apache.hadoop.hbase.filter.Filter;
import org.apache.hadoop.hbase.filter.FilterList;
import org.apache.hadoop.hbase.filter.ParseFilter;
@ -144,7 +144,7 @@ public class TableResource extends ResourceBase {
+ " End Row => " + endRow + " Columns => " + column + " Start Time => " + startTime
+ " End Time => " + endTime + " Cache Blocks => " + cacheBlocks + " Max Versions => "
+ maxVersions + " Batch Size => " + batchSize);
HTableInterface hTable = RESTServlet.getInstance().getTable(this.table);
Table hTable = RESTServlet.getInstance().getTable(this.table);
Scan tableScan = new Scan();
tableScan.setBatch(batchSize);
tableScan.setMaxVersions(maxVersions);

View File

@ -51,6 +51,7 @@ import org.apache.hadoop.hbase.client.Put;
import org.apache.hadoop.hbase.client.Result;
import org.apache.hadoop.hbase.client.ResultScanner;
import org.apache.hadoop.hbase.client.Scan;
import org.apache.hadoop.hbase.client.Table;
import org.apache.hadoop.hbase.exceptions.DeserializationException;
import org.apache.hadoop.hbase.filter.CompareFilter.CompareOp;
import org.apache.hadoop.hbase.filter.QualifierFilter;
@ -172,7 +173,7 @@ public class AccessControlLists {
Bytes.toString(key)+": "+Bytes.toStringBinary(value)
);
}
HTable acls = null;
Table acls = null;
try {
acls = new HTable(conf, ACL_TABLE_NAME);
acls.put(p);
@ -203,7 +204,7 @@ public class AccessControlLists {
LOG.debug("Removing permission "+ userPerm.toString());
}
d.deleteColumns(ACL_LIST_FAMILY, key);
HTable acls = null;
Table acls = null;
try {
acls = new HTable(conf, ACL_TABLE_NAME);
acls.delete(d);
@ -223,7 +224,7 @@ public class AccessControlLists {
LOG.debug("Removing permissions of removed table "+ tableName);
}
HTable acls = null;
Table acls = null;
try {
acls = new HTable(conf, ACL_TABLE_NAME);
acls.delete(d);
@ -243,7 +244,7 @@ public class AccessControlLists {
LOG.debug("Removing permissions of removed namespace "+ namespace);
}
HTable acls = null;
Table acls = null;
try {
acls = new HTable(conf, ACL_TABLE_NAME);
acls.delete(d);
@ -263,7 +264,7 @@ public class AccessControlLists {
" from table "+ tableName);
}
HTable acls = null;
Table acls = null;
try {
acls = new HTable(conf, ACL_TABLE_NAME);
@ -424,7 +425,7 @@ public class AccessControlLists {
Scan scan = new Scan();
scan.addFamily(ACL_LIST_FAMILY);
HTable acls = null;
Table acls = null;
ResultScanner scanner = null;
try {
acls = new HTable(conf, ACL_TABLE_NAME);
@ -467,7 +468,7 @@ public class AccessControlLists {
// for normal user tables, we just read the table row from _acl_
ListMultimap<String, TablePermission> perms = ArrayListMultimap.create();
HTable acls = null;
Table acls = null;
try {
acls = new HTable(conf, ACL_TABLE_NAME);
Get get = new Get(entryName);

View File

@ -30,6 +30,7 @@ import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.hbase.HConstants;
import org.apache.hadoop.hbase.TableName;
import org.apache.hadoop.hbase.client.HTable;
import org.apache.hadoop.hbase.client.Table;
import org.apache.hadoop.hbase.ipc.CoprocessorRpcChannel;
import org.apache.hadoop.hbase.protobuf.ProtobufUtil;
import org.apache.hadoop.hbase.protobuf.generated.AuthenticationProtos;
@ -53,7 +54,7 @@ public class TokenUtil {
*/
public static Token<AuthenticationTokenIdentifier> obtainToken(
Configuration conf) throws IOException {
HTable meta = null;
Table meta = null;
try {
meta = new HTable(conf, TableName.META_TABLE_NAME);
CoprocessorRpcChannel rpcChannel = meta.coprocessorService(HConstants.EMPTY_START_ROW);

View File

@ -49,6 +49,7 @@ import org.apache.hadoop.hbase.client.HBaseAdmin;
import org.apache.hadoop.hbase.client.HTable;
import org.apache.hadoop.hbase.client.ResultScanner;
import org.apache.hadoop.hbase.client.Scan;
import org.apache.hadoop.hbase.client.Table;
import org.apache.hadoop.util.Tool;
import org.apache.hadoop.util.ToolRunner;
@ -326,7 +327,7 @@ public final class Canary implements Tool {
public static abstract class Monitor implements Runnable {
protected Configuration config;
protected HBaseAdmin admin;
protected Admin admin;
protected String[] targets;
protected boolean useRegExp;
protected boolean initialized = false;
@ -476,7 +477,7 @@ public final class Canary implements Tool {
*/
private static void sniff(final Admin admin, final Sink sink, HTableDescriptor tableDesc)
throws Exception {
HTable table = null;
Table table = null;
try {
table = new HTable(admin.getConfiguration(), tableDesc.getName());
@ -506,7 +507,7 @@ public final class Canary implements Tool {
final Admin admin,
final Sink sink,
HRegionInfo region,
HTable table) throws Exception {
Table table) throws Exception {
HTableDescriptor tableDesc = table.getTableDescriptor();
byte[] startKey = null;
Get get = null;
@ -607,7 +608,7 @@ public final class Canary implements Tool {
String serverName = null;
String tableName = null;
HRegionInfo region = null;
HTable table = null;
Table table = null;
Get get = null;
byte[] startKey = null;
Scan scan = null;

View File

@ -73,6 +73,7 @@ import org.apache.hadoop.hbase.ServerName;
import org.apache.hadoop.hbase.TableName;
import org.apache.hadoop.hbase.ZooKeeperConnectionException;
import org.apache.hadoop.hbase.MetaTableAccessor;
import org.apache.hadoop.hbase.client.Admin;
import org.apache.hadoop.hbase.client.Delete;
import org.apache.hadoop.hbase.client.Get;
import org.apache.hadoop.hbase.client.HBaseAdmin;
@ -87,6 +88,7 @@ import org.apache.hadoop.hbase.client.MetaScanner.MetaScannerVisitorBase;
import org.apache.hadoop.hbase.client.Put;
import org.apache.hadoop.hbase.client.Result;
import org.apache.hadoop.hbase.client.RowMutations;
import org.apache.hadoop.hbase.client.Table;
import org.apache.hadoop.hbase.io.hfile.CacheConfig;
import org.apache.hadoop.hbase.io.hfile.HFile;
import org.apache.hadoop.hbase.master.MasterFileSystem;
@ -184,8 +186,8 @@ public class HBaseFsck extends Configured {
private static final Log LOG = LogFactory.getLog(HBaseFsck.class.getName());
private ClusterStatus status;
private HConnection connection;
private HBaseAdmin admin;
private HTable meta;
private Admin admin;
private Table meta;
// threads to do ||izable tasks: retrieve data from regionservers, handle overlapping regions
protected ExecutorService executor;
private long startMillis = System.currentTimeMillis();
@ -2805,7 +2807,7 @@ public class HBaseFsck extends Configured {
HTableDescriptor[] getHTableDescriptors(List<TableName> tableNames) {
HTableDescriptor[] htd = new HTableDescriptor[0];
HBaseAdmin admin = null;
Admin admin = null;
try {
LOG.info("getHTableDescriptors == tableNames => " + tableNames);
admin = new HBaseAdmin(getConf());

View File

@ -36,10 +36,10 @@ import org.apache.hadoop.hbase.TableName;
import org.apache.hadoop.hbase.ZooKeeperConnectionException;
import org.apache.hadoop.hbase.MetaTableAccessor;
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.HTable;
import org.apache.hadoop.hbase.client.Put;
import org.apache.hadoop.hbase.client.Table;
import org.apache.hadoop.hbase.master.RegionState;
import org.apache.hadoop.hbase.protobuf.ProtobufUtil;
import org.apache.hadoop.hbase.protobuf.generated.AdminProtos.AdminService;
@ -63,7 +63,7 @@ public class HBaseFsckRepair {
* @param region Region to undeploy
* @param servers list of Servers to undeploy from
*/
public static void fixMultiAssignment(HBaseAdmin admin, HRegionInfo region,
public static void fixMultiAssignment(Admin admin, HRegionInfo region,
List<ServerName> servers)
throws IOException, KeeperException, InterruptedException {
HRegionInfo actualRegion = new HRegionInfo(region);
@ -89,7 +89,7 @@ public class HBaseFsckRepair {
* @throws IOException
* @throws KeeperException
*/
public static void fixUnassigned(HBaseAdmin admin, HRegionInfo region)
public static void fixUnassigned(Admin admin, HRegionInfo region)
throws IOException, KeeperException {
HRegionInfo actualRegion = new HRegionInfo(region);
@ -109,7 +109,7 @@ public class HBaseFsckRepair {
* side-effect of requiring a HRegionInfo that considers regionId (timestamp)
* in comparators that is addressed by HBASE-5563.
*/
private static void forceOfflineInZK(HBaseAdmin admin, final HRegionInfo region)
private static void forceOfflineInZK(Admin admin, final HRegionInfo region)
throws ZooKeeperConnectionException, KeeperException, IOException {
admin.assign(region.getRegionName());
}
@ -117,7 +117,7 @@ public class HBaseFsckRepair {
/*
* Should we check all assignments or just not in RIT?
*/
public static void waitUntilAssigned(HBaseAdmin admin,
public static void waitUntilAssigned(Admin admin,
HRegionInfo region) throws IOException, InterruptedException {
long timeout = admin.getConfiguration().getLong("hbase.hbck.assign.timeout", 120000);
long expiration = timeout + System.currentTimeMillis();
@ -179,7 +179,7 @@ public class HBaseFsckRepair {
*/
public static void fixMetaHoleOnlineAndAddReplicas(Configuration conf,
HRegionInfo hri, Collection<ServerName> servers, int numReplicas) throws IOException {
HTable meta = new HTable(conf, TableName.META_TABLE_NAME);
Table meta = new HTable(conf, TableName.META_TABLE_NAME);
Put put = MetaTableAccessor.makePutFromRegionInfo(hri);
if (numReplicas > 1) {
Random r = new Random();

View File

@ -36,6 +36,7 @@ import org.apache.hadoop.hbase.HTableDescriptor;
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.Delete;
import org.apache.hadoop.hbase.client.HBaseAdmin;
import org.apache.hadoop.hbase.client.HConnectable;
@ -44,6 +45,7 @@ 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;
import org.apache.hadoop.hbase.regionserver.HRegion;
import org.apache.hadoop.hbase.regionserver.wal.HLog;
import org.apache.hadoop.hbase.regionserver.wal.HLogFactory;
@ -124,7 +126,7 @@ class HMerge {
throw new IllegalStateException(
"HBase instance must be running to merge a normal table");
}
HBaseAdmin admin = new HBaseAdmin(conf);
Admin admin = new HBaseAdmin(conf);
try {
if (!admin.isTableDisabled(tableName)) {
throw new TableNotDisabledException(tableName);
@ -231,7 +233,7 @@ class HMerge {
/** Instantiated to compact a normal user table */
private static class OnlineMerger extends Merger {
private final TableName tableName;
private final HTable table;
private final Table table;
private final ResultScanner metaScanner;
private HRegionInfo latestRegion;

View File

@ -27,11 +27,11 @@ import org.apache.hadoop.hbase.HRegionInfo;
import org.apache.hadoop.hbase.RegionLoad;
import org.apache.hadoop.hbase.ServerLoad;
import org.apache.hadoop.hbase.ServerName;
import org.apache.hadoop.hbase.client.Admin;
import org.apache.hadoop.hbase.client.HBaseAdmin;
import org.apache.hadoop.hbase.client.HTable;
import java.io.IOException;
import java.text.MessageFormat;
import java.util.Arrays;
import java.util.Collection;
import java.util.Collections;
@ -65,7 +65,7 @@ public class RegionSizeCalculator {
}
/** ctor for unit testing */
RegionSizeCalculator (HTable table, HBaseAdmin admin) throws IOException {
RegionSizeCalculator (HTable table, Admin admin) throws IOException {
try {
if (!enabled(table.getConfiguration())) {

View File

@ -39,6 +39,7 @@ import org.apache.hadoop.hbase.client.Put;
import org.apache.hadoop.hbase.client.Result;
import org.apache.hadoop.hbase.client.ResultScanner;
import org.apache.hadoop.hbase.client.Scan;
import org.apache.hadoop.hbase.client.Table;
import org.apache.hadoop.hbase.regionserver.HRegion;
import org.apache.hadoop.hbase.regionserver.InternalScanner;
import org.apache.hadoop.hbase.util.Bytes;
@ -467,12 +468,12 @@ public abstract class HBaseTestCase extends TestCase {
* A class that makes a {@link Incommon} out of a {@link HTable}
*/
public static class HTableIncommon implements Incommon {
final HTable table;
final Table table;
/**
* @param table
*/
public HTableIncommon(final HTable table) {
public HTableIncommon(final Table table) {
super();
this.table = table;
}

View File

@ -63,11 +63,12 @@ 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.HTableInterface;
import org.apache.hadoop.hbase.client.Put;
import org.apache.hadoop.hbase.client.RegionLocator;
import org.apache.hadoop.hbase.client.Result;
import org.apache.hadoop.hbase.client.ResultScanner;
import org.apache.hadoop.hbase.client.Scan;
import org.apache.hadoop.hbase.client.Table;
import org.apache.hadoop.hbase.fs.HFileSystem;
import org.apache.hadoop.hbase.io.compress.Compression;
import org.apache.hadoop.hbase.io.compress.Compression.Algorithm;
@ -898,7 +899,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
HTable t = new HTable(c, TableName.META_TABLE_NAME);
Table t = new HTable(c, TableName.META_TABLE_NAME);
ResultScanner s = t.getScanner(new Scan());
while (s.next() != null) {
continue;
@ -920,7 +921,7 @@ public class HBaseTestingUtility extends HBaseCommonTestingUtility {
public void restartHBaseCluster(int servers) throws IOException, InterruptedException {
this.hbaseCluster = new MiniHBaseCluster(this.conf, servers);
// Don't leave here till we've done a successful scan of the hbase:meta
HTable t = new HTable(new Configuration(this.conf), TableName.META_TABLE_NAME);
Table t = new HTable(new Configuration(this.conf), TableName.META_TABLE_NAME);
ResultScanner s = t.getScanner(new Scan());
while (s.next() != null) {
// do nothing
@ -1912,7 +1913,7 @@ public class HBaseTestingUtility extends HBaseCommonTestingUtility {
return rowCount;
}
public void loadNumericRows(final HTableInterface t, final byte[] f, int startRow, int endRow)
public void loadNumericRows(final Table t, final byte[] f, int startRow, int endRow)
throws IOException {
for (int i = startRow; i < endRow; i++) {
byte[] data = Bytes.toBytes(String.valueOf(i));
@ -1937,7 +1938,7 @@ public class HBaseTestingUtility extends HBaseCommonTestingUtility {
}
}
public void deleteNumericRows(final HTable t, final byte[] f, int startRow, int endRow)
public void deleteNumericRows(final Table t, final byte[] f, int startRow, int endRow)
throws IOException {
for (int i = startRow; i < endRow; i++) {
byte[] data = Bytes.toBytes(String.valueOf(i));
@ -1950,7 +1951,7 @@ public class HBaseTestingUtility extends HBaseCommonTestingUtility {
/**
* Return the number of rows in the given table.
*/
public int countRows(final HTable table) throws IOException {
public int countRows(final Table table) throws IOException {
Scan scan = new Scan();
ResultScanner results = table.getScanner(scan);
int count = 0;
@ -1961,7 +1962,7 @@ public class HBaseTestingUtility extends HBaseCommonTestingUtility {
return count;
}
public int countRows(final HTable table, final byte[]... families) throws IOException {
public int countRows(final Table table, final byte[]... families) throws IOException {
Scan scan = new Scan();
for (byte[] family: families) {
scan.addFamily(family);
@ -1978,7 +1979,7 @@ public class HBaseTestingUtility extends HBaseCommonTestingUtility {
/**
* Return an md5 digest of the entire contents of a table.
*/
public String checksumRows(final HTable table) throws Exception {
public String checksumRows(final Table table) throws Exception {
Scan scan = new Scan();
ResultScanner results = table.getScanner(scan);
MessageDigest digest = MessageDigest.getInstance("MD5");
@ -2084,7 +2085,7 @@ public class HBaseTestingUtility extends HBaseCommonTestingUtility {
final byte[] columnFamily, byte [][] startKeys)
throws IOException {
Arrays.sort(startKeys, Bytes.BYTES_COMPARATOR);
HTable meta = new HTable(c, TableName.META_TABLE_NAME);
Table meta = new HTable(c, TableName.META_TABLE_NAME);
HTableDescriptor htd = table.getTableDescriptor();
if(!htd.hasFamily(columnFamily)) {
HColumnDescriptor hcd = new HColumnDescriptor(columnFamily);
@ -2149,7 +2150,7 @@ public class HBaseTestingUtility extends HBaseCommonTestingUtility {
public List<HRegionInfo> createMultiRegionsInMeta(final Configuration conf,
final HTableDescriptor htd, byte [][] startKeys)
throws IOException {
HTable meta = new HTable(conf, TableName.META_TABLE_NAME);
Table meta = new HTable(conf, TableName.META_TABLE_NAME);
Arrays.sort(startKeys, Bytes.BYTES_COMPARATOR);
List<HRegionInfo> newRegions = new ArrayList<HRegionInfo>(startKeys.length);
// add custom ones
@ -2172,7 +2173,7 @@ public class HBaseTestingUtility extends HBaseCommonTestingUtility {
*/
public List<byte[]> getMetaTableRows() throws IOException {
// TODO: Redo using MetaTableAccessor class
HTable t = new HTable(new Configuration(this.conf), TableName.META_TABLE_NAME);
Table t = new HTable(new Configuration(this.conf), TableName.META_TABLE_NAME);
List<byte[]> rows = new ArrayList<byte[]>();
ResultScanner s = t.getScanner(new Scan());
for (Result result : s) {
@ -2192,7 +2193,7 @@ public class HBaseTestingUtility extends HBaseCommonTestingUtility {
*/
public List<byte[]> getMetaTableRows(TableName tableName) throws IOException {
// TODO: Redo using MetaTableAccessor.
HTable t = new HTable(new Configuration(this.conf), TableName.META_TABLE_NAME);
Table t = new HTable(new Configuration(this.conf), TableName.META_TABLE_NAME);
List<byte[]> rows = new ArrayList<byte[]>();
ResultScanner s = t.getScanner(new Scan());
for (Result result : s) {
@ -2638,7 +2639,7 @@ public class HBaseTestingUtility extends HBaseCommonTestingUtility {
* @param table The table to find the region.
* @throws IOException
*/
public void closeRegionByRow(String row, HTable table) throws IOException {
public void closeRegionByRow(String row, RegionLocator table) throws IOException {
closeRegionByRow(Bytes.toBytes(row), table);
}
@ -2649,7 +2650,7 @@ public class HBaseTestingUtility extends HBaseCommonTestingUtility {
* @param table The table to find the region.
* @throws IOException
*/
public void closeRegionByRow(byte[] row, HTable table) throws IOException {
public void closeRegionByRow(byte[] row, RegionLocator table) throws IOException {
HRegionLocation hrl = table.getRegionLocation(row);
closeRegion(hrl.getRegionInfo().getRegionName());
}
@ -2975,7 +2976,7 @@ public class HBaseTestingUtility extends HBaseCommonTestingUtility {
*/
public void waitUntilAllRegionsAssigned(final TableName tableName, final long timeout)
throws IOException {
final HTable meta = new HTable(getConfiguration(), TableName.META_TABLE_NAME);
final Table meta = new HTable(getConfiguration(), TableName.META_TABLE_NAME);
try {
waitFor(timeout, 200, true, new Predicate<IOException>() {
@Override
@ -3371,7 +3372,7 @@ public class HBaseTestingUtility extends HBaseCommonTestingUtility {
}
public static int getMetaRSPort(Configuration conf) throws IOException {
HTable table = new HTable(conf, TableName.META_TABLE_NAME);
RegionLocator table = new HTable(conf, TableName.META_TABLE_NAME);
HRegionLocation hloc = table.getRegionLocation(Bytes.toBytes(""));
table.close();
return hloc.getPort();

View File

@ -55,11 +55,11 @@ 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.HTableInterface;
import org.apache.hadoop.hbase.client.Put;
import org.apache.hadoop.hbase.client.Result;
import org.apache.hadoop.hbase.client.ResultScanner;
import org.apache.hadoop.hbase.client.Scan;
import org.apache.hadoop.hbase.client.Table;
import org.apache.hadoop.hbase.filter.BinaryComparator;
import org.apache.hadoop.hbase.filter.CompareFilter;
import org.apache.hadoop.hbase.filter.Filter;
@ -923,7 +923,7 @@ public class PerformanceEvaluation extends Configured implements Tool {
private final Sampler<?> traceSampler;
private final SpanReceiverHost receiverHost;
protected HConnection connection;
protected HTableInterface table;
protected Table table;
private String testName;
private Histogram latency;
@ -1567,7 +1567,7 @@ public class PerformanceEvaluation extends Configured implements Tool {
// Log the configuration we're going to run with. Uses JSON mapper because lazy. It'll do
// the TestOptions introspection for us and dump the output in a readable format.
LOG.info(cmd.getSimpleName() + " test run options=" + MAPPER.writeValueAsString(opts));
HBaseAdmin admin = null;
Admin admin = null;
try {
admin = new HBaseAdmin(getConf());
checkTable(admin, opts);

View File

@ -30,6 +30,7 @@ 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.Scan;
import org.apache.hadoop.hbase.client.Table;
import org.apache.hadoop.hbase.client.TableSnapshotScanner;
import org.apache.hadoop.hbase.client.metrics.ScanMetrics;
import org.apache.hadoop.hbase.io.ImmutableBytesWritable;
@ -146,7 +147,7 @@ public class ScanPerformanceEvaluation extends AbstractHBaseTool {
Stopwatch scanTimer = new Stopwatch();
tableOpenTimer.start();
HTable table = new HTable(getConf(), TableName.valueOf(tablename));
Table table = new HTable(getConf(), TableName.valueOf(tablename));
tableOpenTimer.stop();
Scan scan = getScan();

View File

@ -35,6 +35,7 @@ import org.apache.hadoop.hbase.client.Put;
import org.apache.hadoop.hbase.client.Result;
import org.apache.hadoop.hbase.client.ResultScanner;
import org.apache.hadoop.hbase.client.Scan;
import org.apache.hadoop.hbase.client.Table;
import org.apache.hadoop.hbase.regionserver.ConstantSizeRegionSplitPolicy;
import org.apache.hadoop.hbase.util.Bytes;
import org.apache.hadoop.util.StringUtils;
@ -97,7 +98,7 @@ public class TestAcidGuarantees implements Tool {
byte data[] = new byte[10];
byte targetRows[][];
byte targetFamilies[][];
HTable table;
Table table;
AtomicLong numWritten = new AtomicLong();
public AtomicityWriter(TestContext ctx, byte targetRows[][],
@ -131,7 +132,7 @@ public class TestAcidGuarantees implements Tool {
public static class AtomicGetReader extends RepeatingTestThread {
byte targetRow[];
byte targetFamilies[][];
HTable table;
Table table;
int numVerified = 0;
AtomicLong numRead = new AtomicLong();
@ -189,7 +190,7 @@ public class TestAcidGuarantees implements Tool {
*/
public static class AtomicScanReader extends RepeatingTestThread {
byte targetFamilies[][];
HTable table;
Table table;
AtomicLong numScans = new AtomicLong();
AtomicLong numRowsScanned = new AtomicLong();

View File

@ -28,9 +28,9 @@ import org.apache.commons.logging.LogFactory;
import org.apache.hadoop.fs.FileSystem;
import org.apache.hadoop.fs.Path;
import org.apache.hadoop.hbase.client.Get;
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;
import org.apache.hadoop.hbase.util.Bytes;
import org.apache.hadoop.hbase.zookeeper.MiniZooKeeperCluster;
import org.apache.hadoop.hdfs.MiniDFSCluster;
@ -87,8 +87,8 @@ public class TestHBaseTestingUtility {
final byte[] QUAL_NAME = Bytes.toBytes("qual");
final byte[] VALUE = Bytes.toBytes("value");
HTable table1 = htu1.createTable(TABLE_NAME, FAM_NAME);
HTable table2 = htu2.createTable(TABLE_NAME, FAM_NAME);
Table table1 = htu1.createTable(TABLE_NAME, FAM_NAME);
Table table2 = htu2.createTable(TABLE_NAME, FAM_NAME);
Put put = new Put(ROW);
put.add(FAM_NAME, QUAL_NAME, VALUE);

View File

@ -36,6 +36,7 @@ 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.Table;
import org.apache.hadoop.hbase.util.Bytes;
import org.apache.hadoop.hbase.util.Pair;
import org.junit.AfterClass;
@ -333,7 +334,7 @@ public class TestMetaTableAccessor {
long seqNum100 = random.nextLong();
HTable meta = MetaTableAccessor.getMetaHTable(hConnection);
Table meta = MetaTableAccessor.getMetaHTable(hConnection);
try {
MetaTableAccessor.updateRegionLocation(hConnection, primary, serverName0, seqNum0);
@ -360,7 +361,7 @@ public class TestMetaTableAccessor {
}
}
public static void assertMetaLocation(HTable meta, byte[] row, ServerName serverName,
public static void assertMetaLocation(Table meta, byte[] row, ServerName serverName,
long seqNum, int replicaId, boolean checkSeqNum) throws IOException {
Get get = new Get(row);
Result result = meta.get(get);

View File

@ -33,6 +33,7 @@ import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.hbase.HBaseTestCase.FlushCache;
import org.apache.hadoop.hbase.HBaseTestCase.HTableIncommon;
import org.apache.hadoop.hbase.HBaseTestCase.Incommon;
import org.apache.hadoop.hbase.client.Admin;
import org.apache.hadoop.hbase.client.Get;
import org.apache.hadoop.hbase.client.HBaseAdmin;
import org.apache.hadoop.hbase.client.HTable;
@ -40,6 +41,7 @@ import org.apache.hadoop.hbase.client.Put;
import org.apache.hadoop.hbase.client.Result;
import org.apache.hadoop.hbase.client.ResultScanner;
import org.apache.hadoop.hbase.client.Scan;
import org.apache.hadoop.hbase.client.Table;
import org.apache.hadoop.hbase.util.Bytes;
import org.junit.After;
import org.junit.AfterClass;
@ -56,7 +58,7 @@ import org.junit.experimental.categories.Category;
public class TestMultiVersions {
private static final Log LOG = LogFactory.getLog(TestMultiVersions.class);
private static final HBaseTestingUtility UTIL = new HBaseTestingUtility();
private HBaseAdmin admin;
private Admin admin;
private static final int NUM_SLAVES = 3;
@ -97,7 +99,7 @@ public class TestMultiVersions {
hcd.setMaxVersions(3);
desc.addFamily(hcd);
this.admin.createTable(desc);
HTable table = new HTable(UTIL.getConfiguration(), desc.getTableName());
Table table = new HTable(UTIL.getConfiguration(), desc.getTableName());
// TODO: Remove these deprecated classes or pull them in here if this is
// only test using them.
Incommon incommon = new HTableIncommon(table);
@ -140,7 +142,7 @@ public class TestMultiVersions {
this.admin.createTable(desc);
Put put = new Put(row, timestamp1);
put.add(contents, contents, value1);
HTable table = new HTable(UTIL.getConfiguration(), tableName);
Table table = new HTable(UTIL.getConfiguration(), tableName);
table.put(put);
// Shut down and restart the HBase cluster
table.close();

View File

@ -37,6 +37,7 @@ import org.apache.hadoop.hbase.client.Admin;
import org.apache.hadoop.hbase.client.Get;
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.master.HMaster;
import org.apache.hadoop.hbase.util.Bytes;
import org.apache.hadoop.hbase.util.FSUtils;
@ -249,7 +250,7 @@ public class TestNamespace {
}
//sanity check try to write and read from table
HTable table = new HTable(TEST_UTIL.getConfiguration(), desc.getTableName());
Table table = new HTable(TEST_UTIL.getConfiguration(), desc.getTableName());
Put p = new Put(Bytes.toBytes("row1"));
p.add(Bytes.toBytes("my_cf"),Bytes.toBytes("my_col"),Bytes.toBytes("value1"));
table.put(p);

View File

@ -29,8 +29,10 @@ import java.util.List;
import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
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.RegionLocator;
import org.apache.hadoop.hbase.master.RegionStates;
import org.apache.hadoop.hbase.protobuf.ProtobufUtil;
import org.apache.hadoop.hbase.regionserver.HRegionServer;
@ -63,7 +65,7 @@ public class TestRegionRebalancing {
private static final byte[] FAMILY_NAME = Bytes.toBytes("col");
public static final Log LOG = LogFactory.getLog(TestRegionRebalancing.class);
private final HBaseTestingUtility UTIL = new HBaseTestingUtility();
private HTable table;
private RegionLocator table;
private HTableDescriptor desc;
private String balancerName;
@ -94,7 +96,7 @@ public class TestRegionRebalancing {
@Test (timeout=300000)
public void testRebalanceOnRegionServerNumberChange()
throws IOException, InterruptedException {
HBaseAdmin admin = new HBaseAdmin(UTIL.getConfiguration());
Admin admin = new HBaseAdmin(UTIL.getConfiguration());
admin.createTable(this.desc, Arrays.copyOfRange(HBaseTestingUtility.KEYS,
1, HBaseTestingUtility.KEYS.length));
this.table = new HTable(UTIL.getConfiguration(), this.desc.getTableName());

View File

@ -34,6 +34,7 @@ import java.util.Map;
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.Get;
import org.apache.hadoop.hbase.client.HBaseAdmin;
import org.apache.hadoop.hbase.client.HConnection;
@ -43,6 +44,7 @@ import org.apache.hadoop.hbase.client.Put;
import org.apache.hadoop.hbase.client.Result;
import org.apache.hadoop.hbase.client.ResultScanner;
import org.apache.hadoop.hbase.client.Scan;
import org.apache.hadoop.hbase.client.Table;
import org.apache.hadoop.hbase.master.HMaster;
import org.apache.hadoop.hbase.master.LoadBalancer;
import org.apache.hadoop.hbase.master.balancer.SimpleLoadBalancer;
@ -249,14 +251,14 @@ public class TestZooKeeper {
HColumnDescriptor family = new HColumnDescriptor("fam");
desc.addFamily(family);
LOG.info("Creating table " + tableName);
HBaseAdmin admin = new HBaseAdmin(TEST_UTIL.getConfiguration());
Admin admin = new HBaseAdmin(TEST_UTIL.getConfiguration());
try {
admin.createTable(desc);
} finally {
admin.close();
}
HTable table =
Table table =
new HTable(new Configuration(TEST_UTIL.getConfiguration()), tableName);
Put put = new Put(Bytes.toBytes("testrow"));
put.add(Bytes.toBytes("fam"),
@ -269,11 +271,11 @@ public class TestZooKeeper {
@Test
public void testMultipleZK()
throws IOException, NoSuchMethodException, InvocationTargetException, IllegalAccessException {
HTable localMeta =
Table localMeta =
new HTable(new Configuration(TEST_UTIL.getConfiguration()), TableName.META_TABLE_NAME);
Configuration otherConf = new Configuration(TEST_UTIL.getConfiguration());
otherConf.set(HConstants.ZOOKEEPER_QUORUM, "127.0.0.1");
HTable ipMeta = new HTable(otherConf, TableName.META_TABLE_NAME);
Table ipMeta = new HTable(otherConf, TableName.META_TABLE_NAME);
// dummy, just to open the connection
final byte [] row = new byte [] {'r'};
@ -490,7 +492,7 @@ public class TestZooKeeper {
ZooKeeperWatcher zkw = m.getZooKeeper();
int expectedNumOfListeners = zkw.getNumberOfListeners();
// now the cluster is up. So assign some regions.
HBaseAdmin admin = new HBaseAdmin(TEST_UTIL.getConfiguration());
Admin admin = new HBaseAdmin(TEST_UTIL.getConfiguration());
try {
byte[][] SPLIT_KEYS = new byte[][] { Bytes.toBytes("a"), Bytes.toBytes("b"),
Bytes.toBytes("c"), Bytes.toBytes("d"), Bytes.toBytes("e"), Bytes.toBytes("f"),
@ -527,8 +529,8 @@ public class TestZooKeeper {
cluster.startRegionServer();
HMaster m = cluster.getMaster();
// now the cluster is up. So assign some regions.
HBaseAdmin admin = new HBaseAdmin(TEST_UTIL.getConfiguration());
HTable table = null;
Admin admin = new HBaseAdmin(TEST_UTIL.getConfiguration());
Table table = null;
try {
byte[][] SPLIT_KEYS = new byte[][] { Bytes.toBytes("1"), Bytes.toBytes("2"),
Bytes.toBytes("3"), Bytes.toBytes("4"), Bytes.toBytes("5") };

View File

@ -246,7 +246,7 @@ public class TestAdmin {
final byte [] qualifier = Bytes.toBytes("qualifier");
final byte [] value = Bytes.toBytes("value");
final TableName table = TableName.valueOf("testDisableAndEnableTable");
HTable ht = TEST_UTIL.createTable(table, HConstants.CATALOG_FAMILY);
Table ht = TEST_UTIL.createTable(table, HConstants.CATALOG_FAMILY);
Put put = new Put(row);
put.add(HConstants.CATALOG_FAMILY, qualifier, value);
ht.put(put);
@ -291,8 +291,8 @@ public class TestAdmin {
final byte [] value = Bytes.toBytes("value");
final byte [] table1 = Bytes.toBytes("testDisableAndEnableTable1");
final byte [] table2 = Bytes.toBytes("testDisableAndEnableTable2");
HTable ht1 = TEST_UTIL.createTable(table1, HConstants.CATALOG_FAMILY);
HTable ht2 = TEST_UTIL.createTable(table2, HConstants.CATALOG_FAMILY);
Table ht1 = TEST_UTIL.createTable(table1, HConstants.CATALOG_FAMILY);
Table ht2 = TEST_UTIL.createTable(table2, HConstants.CATALOG_FAMILY);
Put put = new Put(row);
put.add(HConstants.CATALOG_FAMILY, qualifier, value);
ht1.put(put);
@ -400,7 +400,7 @@ public class TestAdmin {
htd.addFamily(fam2);
htd.addFamily(fam3);
this.admin.createTable(htd);
HTable table = new HTable(TEST_UTIL.getConfiguration(), "myTestTable");
Table table = new HTable(TEST_UTIL.getConfiguration(), "myTestTable");
HTableDescriptor confirmedHtd = table.getTableDescriptor();
assertEquals(htd.compareTo(confirmedHtd), 0);
table.close();
@ -816,7 +816,7 @@ public class TestAdmin {
TableName TABLE_4 = TableName.valueOf(tableName.getNameAsString() + "_4");
desc = new HTableDescriptor(TABLE_4);
desc.addFamily(new HColumnDescriptor(HConstants.CATALOG_FAMILY));
HBaseAdmin ladmin = new HBaseAdmin(TEST_UTIL.getConfiguration());
Admin ladmin = new HBaseAdmin(TEST_UTIL.getConfiguration());
try {
ladmin.createTable(desc, splitKeys);
assertTrue("Should not be able to create this table because of " +
@ -1238,7 +1238,7 @@ public class TestAdmin {
// 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 };
HBaseAdmin hbaseadmin = new HBaseAdmin(TEST_UTIL.getConfiguration());
Admin hbaseadmin = new HBaseAdmin(TEST_UTIL.getConfiguration());
HTableDescriptor htd = new HTableDescriptor(TableName.valueOf(name));
htd.addFamily(new HColumnDescriptor(HConstants.CATALOG_FAMILY));
hbaseadmin.createTable(htd, startKey, endKey, expectedRegions);
@ -1255,7 +1255,7 @@ public class TestAdmin {
@Test (timeout=300000)
public void testReadOnlyTable() throws Exception {
byte [] name = Bytes.toBytes("testReadOnlyTable");
HTable table = TEST_UTIL.createTable(name, HConstants.CATALOG_FAMILY);
Table table = TEST_UTIL.createTable(name, HConstants.CATALOG_FAMILY);
byte[] value = Bytes.toBytes("somedata");
// This used to use an empty row... That must have been a bug
Put put = new Put(value);
@ -1324,7 +1324,7 @@ public class TestAdmin {
@Test (expected=TableNotDisabledException.class, timeout=300000)
public void testTableNotDisabledExceptionWithATable() throws IOException {
final TableName name = TableName.valueOf("testTableNotDisabledExceptionWithATable");
HTable t = TEST_UTIL.createTable(name, HConstants.CATALOG_FAMILY);
Table t = TEST_UTIL.createTable(name, HConstants.CATALOG_FAMILY);
try {
this.admin.enableTable(name);
}finally {
@ -1338,7 +1338,7 @@ public class TestAdmin {
*/
@Test (expected=TableNotFoundException.class, timeout=300000)
public void testTableNotFoundExceptionWithoutAnyTables() throws IOException {
HTable ht =
Table ht =
new HTable(TEST_UTIL.getConfiguration(),"testTableNotFoundExceptionWithoutAnyTables");
ht.get(new Get("e".getBytes()));
}
@ -1659,7 +1659,7 @@ public class TestAdmin {
HTableDescriptor desc = new HTableDescriptor(TableName.valueOf(tableName));
desc.addFamily(new HColumnDescriptor(HConstants.CATALOG_FAMILY));
admin.createTable(desc);
HTable table = new HTable(TEST_UTIL.getConfiguration(), tableName);
Table table = new HTable(TEST_UTIL.getConfiguration(), tableName);
HRegionServer regionServer = TEST_UTIL.getRSForFirstRegionInTable(Bytes.toBytes(tableName));
for (int i = 1; i <= 256; i++) { // 256 writes should cause 8 log rolls

View File

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

View File

@ -84,7 +84,7 @@ public class TestClientScannerRPCTimeout {
@Test
public void testScannerNextRPCTimesout() throws Exception {
final byte[] TABLE_NAME = Bytes.toBytes("testScannerNextRPCTimesout");
HTable ht = TEST_UTIL.createTable(TABLE_NAME, FAMILY);
Table ht = TEST_UTIL.createTable(TABLE_NAME, FAMILY);
byte[] r1 = Bytes.toBytes("row-1");
byte[] r2 = Bytes.toBytes("row-2");
byte[] r3 = Bytes.toBytes("row-3");
@ -123,7 +123,7 @@ public class TestClientScannerRPCTimeout {
RSRpcServicesWithScanTimeout.tryNumber <= CLIENT_RETRIES_NUMBER);
}
private void putToTable(HTable ht, byte[] rowkey) throws IOException {
private void putToTable(Table ht, byte[] rowkey) throws IOException {
Put put = new Put(rowkey);
put.add(FAMILY, QUALIFIER, VALUE);
ht.put(put);

View File

@ -87,7 +87,7 @@ 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;
try {
admin = new HBaseAdmin(conf);
HConnection connection = admin.getConnection();

View File

@ -184,7 +184,7 @@ public class TestFromClientSide {
desc.addFamily(hcd);
TEST_UTIL.getHBaseAdmin().createTable(desc);
Configuration c = TEST_UTIL.getConfiguration();
HTable h = new HTable(c, TABLENAME);
Table h = new HTable(c, TABLENAME);
long ts = System.currentTimeMillis();
Put p = new Put(T1, ts);
@ -244,7 +244,7 @@ public class TestFromClientSide {
final byte[] COLUMN = Bytes.toBytes("column");
final byte[] VALUE = Bytes.toBytes("value");
HTable table = TEST_UTIL.createTable(TABLENAME, FAMILY);
Table table = TEST_UTIL.createTable(TABLENAME, FAMILY);
// future timestamp
long ts = System.currentTimeMillis() * 2;
@ -386,7 +386,7 @@ public class TestFromClientSide {
byte[] TABLE = Bytes.toBytes("testGetConfiguration");
byte[][] FAMILIES = new byte[][] { Bytes.toBytes("foo") };
Configuration conf = TEST_UTIL.getConfiguration();
HTable table = TEST_UTIL.createTable(TABLE, FAMILIES, conf);
Table table = TEST_UTIL.createTable(TABLE, FAMILIES, conf);
assertSame(conf, table.getConfiguration());
}
@ -415,7 +415,7 @@ public class TestFromClientSide {
putRows(ht, 3, value2, keyPrefix1);
putRows(ht, 3, value2, keyPrefix2);
putRows(ht, 3, value2, keyPrefix3);
HTable table = new HTable(TEST_UTIL.getConfiguration(), TABLE);
Table table = new HTable(TEST_UTIL.getConfiguration(), TABLE);
System.out.println("Checking values for key: " + keyPrefix1);
assertEquals("Got back incorrect number of rows from scan", 3,
getNumberOfRows(keyPrefix1, value2, table));
@ -442,7 +442,7 @@ public class TestFromClientSide {
getNumberOfRows(keyPrefix2, value2, table));
}
private void deleteColumns(HTable ht, String value, String keyPrefix)
private void deleteColumns(Table ht, String value, String keyPrefix)
throws IOException {
ResultScanner scanner = buildScanner(keyPrefix, value, ht);
Iterator<Result> it = scanner.iterator();
@ -457,7 +457,7 @@ public class TestFromClientSide {
assertEquals("Did not perform correct number of deletes", 3, count);
}
private int getNumberOfRows(String keyPrefix, String value, HTable ht)
private int getNumberOfRows(String keyPrefix, String value, Table ht)
throws Exception {
ResultScanner resultScanner = buildScanner(keyPrefix, value, ht);
Iterator<Result> scanner = resultScanner.iterator();
@ -474,7 +474,7 @@ public class TestFromClientSide {
return numberOfResults;
}
private ResultScanner buildScanner(String keyPrefix, String value, HTable ht)
private ResultScanner buildScanner(String keyPrefix, String value, Table ht)
throws IOException {
// OurFilterList allFilters = new OurFilterList();
FilterList allFilters = new FilterList(/* FilterList.Operator.MUST_PASS_ALL */);
@ -500,7 +500,7 @@ public class TestFromClientSide {
return ht.getScanner(scan);
}
private void putRows(HTable ht, int numRows, String value, String key)
private void putRows(Table ht, int numRows, String value, String key)
throws IOException {
for (int i = 0; i < numRows; i++) {
String row = key + "_" + UUID.randomUUID().toString();
@ -614,7 +614,7 @@ public class TestFromClientSide {
* @return Count of rows in table.
* @throws IOException
*/
private int countRows(final HTable t, final Scan s)
private int countRows(final Table t, final Scan s)
throws IOException {
// Assert all rows in table.
ResultScanner scanner = t.getScanner(s);
@ -627,7 +627,7 @@ public class TestFromClientSide {
return count;
}
private void assertRowCount(final HTable t, final int expected)
private void assertRowCount(final Table t, final int expected)
throws IOException {
assertEquals(expected, countRows(t, new Scan()));
}
@ -675,7 +675,7 @@ public class TestFromClientSide {
@Test
public void testSuperSimple() throws Exception {
byte [] TABLE = Bytes.toBytes("testSuperSimple");
HTable ht = TEST_UTIL.createTable(TABLE, FAMILY);
Table ht = TEST_UTIL.createTable(TABLE, FAMILY);
Put put = new Put(ROW);
put.add(FAMILY, QUALIFIER, VALUE);
ht.put(put);
@ -692,7 +692,7 @@ public class TestFromClientSide {
byte [] TABLE = Bytes.toBytes("testMaxKeyValueSize");
Configuration conf = TEST_UTIL.getConfiguration();
String oldMaxSize = conf.get("hbase.client.keyvalue.maxsize");
HTable ht = TEST_UTIL.createTable(TABLE, FAMILY);
Table ht = TEST_UTIL.createTable(TABLE, FAMILY);
byte[] value = new byte[4 * 1024 * 1024];
Put put = new Put(ROW);
put.add(FAMILY, QUALIFIER, value);
@ -712,7 +712,7 @@ public class TestFromClientSide {
@Test
public void testFilters() throws Exception {
byte [] TABLE = Bytes.toBytes("testFilters");
HTable ht = TEST_UTIL.createTable(TABLE, FAMILY);
Table ht = TEST_UTIL.createTable(TABLE, FAMILY);
byte [][] ROWS = makeN(ROW, 10);
byte [][] QUALIFIERS = {
Bytes.toBytes("col0-<d2v1>-<d3v2>"), Bytes.toBytes("col1-<d2v1>-<d3v2>"),
@ -748,7 +748,7 @@ public class TestFromClientSide {
@Test
public void testKeyOnlyFilter() throws Exception {
byte [] TABLE = Bytes.toBytes("testKeyOnlyFilter");
HTable ht = TEST_UTIL.createTable(TABLE, FAMILY);
Table ht = TEST_UTIL.createTable(TABLE, FAMILY);
byte [][] ROWS = makeN(ROW, 10);
byte [][] QUALIFIERS = {
Bytes.toBytes("col0-<d2v1>-<d3v2>"), Bytes.toBytes("col1-<d2v1>-<d3v2>"),
@ -785,7 +785,7 @@ public class TestFromClientSide {
@Test
public void testSimpleMissing() throws Exception {
byte [] TABLE = Bytes.toBytes("testSimpleMissing");
HTable ht = TEST_UTIL.createTable(TABLE, FAMILY);
Table ht = TEST_UTIL.createTable(TABLE, FAMILY);
byte [][] ROWS = makeN(ROW, 4);
// Try to get a row on an empty table
@ -901,7 +901,7 @@ public class TestFromClientSide {
byte [][] QUALIFIERS = makeN(QUALIFIER, 10);
byte [][] VALUES = makeN(VALUE, 10);
HTable ht = TEST_UTIL.createTable(TABLE, FAMILIES);
Table ht = TEST_UTIL.createTable(TABLE, FAMILIES);
Get get;
Scan scan;
@ -1210,7 +1210,7 @@ public class TestFromClientSide {
fail("Creating a table with a null family passed, should fail");
} catch(Exception e) {}
HTable ht = TEST_UTIL.createTable(TABLE, FAMILY);
Table ht = TEST_UTIL.createTable(TABLE, FAMILY);
// Null row (should NOT work)
try {
@ -1309,7 +1309,7 @@ public class TestFromClientSide {
long [] STAMPS = makeStamps(20);
byte [][] VALUES = makeNAscii(VALUE, 20);
HTable ht = TEST_UTIL.createTable(TABLE, FAMILY, 10);
Table ht = TEST_UTIL.createTable(TABLE, FAMILY, 10);
// Insert 4 versions of same column
Put put = new Put(ROW);
@ -1523,7 +1523,7 @@ public class TestFromClientSide {
int [] LIMITS = {1,3,5};
long [] STAMPS = makeStamps(10);
byte [][] VALUES = makeNAscii(VALUE, 10);
HTable ht = TEST_UTIL.createTable(TABLE, FAMILIES, LIMITS);
Table ht = TEST_UTIL.createTable(TABLE, FAMILIES, LIMITS);
// Insert limit + 1 on each family
Put put = new Put(ROW);
@ -1719,7 +1719,7 @@ public class TestFromClientSide {
byte [][] VALUES = makeN(VALUE, 5);
long [] ts = {1000, 2000, 3000, 4000, 5000};
HTable ht = TEST_UTIL.createTable(TABLE, FAMILY, 5);
Table ht = TEST_UTIL.createTable(TABLE, FAMILY, 5);
Put put = new Put(ROW);
for (int q = 0; q < 1; q++)
@ -1758,7 +1758,7 @@ public class TestFromClientSide {
long [] ts = {1000, 2000, 3000, 4000, 5000};
HBaseAdmin admin = new HBaseAdmin(TEST_UTIL.getConfiguration());
HTable ht = TEST_UTIL.createTable(TABLE, FAMILY, 5);
Table ht = TEST_UTIL.createTable(TABLE, FAMILY, 5);
Put put = null;
Result result = null;
Get get = null;
@ -1873,7 +1873,7 @@ public class TestFromClientSide {
byte [][] VALUES = makeN(VALUE, 5);
long [] ts = {1000, 2000, 3000, 4000, 5000};
HTable ht = TEST_UTIL.createTable(TABLE, FAMILIES, 3);
Table ht = TEST_UTIL.createTable(TABLE, FAMILIES, 3);
Put put = new Put(ROW);
put.add(FAMILIES[0], QUALIFIER, ts[0], VALUES[0]);
@ -2240,7 +2240,7 @@ public class TestFromClientSide {
byte [][] ROWS = makeN(ROW, numRows);
byte [][] QUALIFIERS = makeN(QUALIFIER, numColsPerRow);
HTable ht = TEST_UTIL.createTable(TABLE, FAMILY);
Table ht = TEST_UTIL.createTable(TABLE, FAMILY);
// Insert rows
@ -2323,7 +2323,7 @@ public class TestFromClientSide {
byte [][] VALUES = makeNAscii(VALUE, 7);
long [] STAMPS = makeStamps(7);
HTable ht = TEST_UTIL.createTable(TABLE, FAMILY, 10);
Table ht = TEST_UTIL.createTable(TABLE, FAMILY, 10);
// Insert three versions
@ -2387,7 +2387,7 @@ public class TestFromClientSide {
byte [][] VALUES = makeNAscii(VALUE, 7);
long [] STAMPS = makeStamps(7);
HTable ht = TEST_UTIL.createTable(TABLE, FAMILY, 10);
Table ht = TEST_UTIL.createTable(TABLE, FAMILY, 10);
// Insert lots versions
@ -2434,7 +2434,7 @@ public class TestFromClientSide {
byte [] TABLE = Bytes.toBytes("testJiraTest1014");
HTable ht = TEST_UTIL.createTable(TABLE, FAMILY, 10);
Table ht = TEST_UTIL.createTable(TABLE, FAMILY, 10);
long manualStamp = 12345;
@ -2461,7 +2461,7 @@ public class TestFromClientSide {
byte [][] VALUES = makeNAscii(VALUE, 7);
long [] STAMPS = makeStamps(7);
HTable ht = TEST_UTIL.createTable(TABLE, FAMILY, 10);
Table ht = TEST_UTIL.createTable(TABLE, FAMILY, 10);
// Insert lots versions
@ -2504,7 +2504,7 @@ public class TestFromClientSide {
byte [][] VALUES = makeNAscii(VALUE, 7);
long [] STAMPS = makeStamps(7);
HTable ht = TEST_UTIL.createTable(TABLE, FAMILY, 10);
Table ht = TEST_UTIL.createTable(TABLE, FAMILY, 10);
// Insert lots versions
@ -2533,7 +2533,7 @@ public class TestFromClientSide {
// Bulk Testers
//
private void getVersionRangeAndVerifyGreaterThan(HTable ht, byte [] row,
private void getVersionRangeAndVerifyGreaterThan(Table ht, byte [] row,
byte [] family, byte [] qualifier, long [] stamps, byte [][] values,
int start, int end)
throws IOException {
@ -2545,7 +2545,7 @@ public class TestFromClientSide {
assertNResult(result, row, family, qualifier, stamps, values, start+1, end);
}
private void getVersionRangeAndVerify(HTable ht, byte [] row, byte [] family,
private void getVersionRangeAndVerify(Table ht, byte [] row, byte [] family,
byte [] qualifier, long [] stamps, byte [][] values, int start, int end)
throws IOException {
Get get = new Get(row);
@ -2556,7 +2556,7 @@ public class TestFromClientSide {
assertNResult(result, row, family, qualifier, stamps, values, start, end);
}
private void getAllVersionsAndVerify(HTable ht, byte [] row, byte [] family,
private void getAllVersionsAndVerify(Table ht, byte [] row, byte [] family,
byte [] qualifier, long [] stamps, byte [][] values, int start, int end)
throws IOException {
Get get = new Get(row);
@ -2566,7 +2566,7 @@ public class TestFromClientSide {
assertNResult(result, row, family, qualifier, stamps, values, start, end);
}
private void scanVersionRangeAndVerifyGreaterThan(HTable ht, byte [] row,
private void scanVersionRangeAndVerifyGreaterThan(Table ht, byte [] row,
byte [] family, byte [] qualifier, long [] stamps, byte [][] values,
int start, int end)
throws IOException {
@ -2578,7 +2578,7 @@ public class TestFromClientSide {
assertNResult(result, row, family, qualifier, stamps, values, start+1, end);
}
private void scanVersionRangeAndVerify(HTable ht, byte [] row, byte [] family,
private void scanVersionRangeAndVerify(Table ht, byte [] row, byte [] family,
byte [] qualifier, long [] stamps, byte [][] values, int start, int end)
throws IOException {
Scan scan = new Scan(row);
@ -2589,7 +2589,7 @@ public class TestFromClientSide {
assertNResult(result, row, family, qualifier, stamps, values, start, end);
}
private void scanAllVersionsAndVerify(HTable ht, byte [] row, byte [] family,
private void scanAllVersionsAndVerify(Table ht, byte [] row, byte [] family,
byte [] qualifier, long [] stamps, byte [][] values, int start, int end)
throws IOException {
Scan scan = new Scan(row);
@ -2599,7 +2599,7 @@ public class TestFromClientSide {
assertNResult(result, row, family, qualifier, stamps, values, start, end);
}
private void getVersionAndVerify(HTable ht, byte [] row, byte [] family,
private void getVersionAndVerify(Table ht, byte [] row, byte [] family,
byte [] qualifier, long stamp, byte [] value)
throws Exception {
Get get = new Get(row);
@ -2610,7 +2610,7 @@ public class TestFromClientSide {
assertSingleResult(result, row, family, qualifier, stamp, value);
}
private void getVersionAndVerifyMissing(HTable ht, byte [] row, byte [] family,
private void getVersionAndVerifyMissing(Table ht, byte [] row, byte [] family,
byte [] qualifier, long stamp)
throws Exception {
Get get = new Get(row);
@ -2621,7 +2621,7 @@ public class TestFromClientSide {
assertEmptyResult(result);
}
private void scanVersionAndVerify(HTable ht, byte [] row, byte [] family,
private void scanVersionAndVerify(Table ht, byte [] row, byte [] family,
byte [] qualifier, long stamp, byte [] value)
throws Exception {
Scan scan = new Scan(row);
@ -2632,7 +2632,7 @@ public class TestFromClientSide {
assertSingleResult(result, row, family, qualifier, stamp, value);
}
private void scanVersionAndVerifyMissing(HTable ht, byte [] row,
private void scanVersionAndVerifyMissing(Table ht, byte [] row,
byte [] family, byte [] qualifier, long stamp)
throws Exception {
Scan scan = new Scan(row);
@ -2643,7 +2643,7 @@ public class TestFromClientSide {
assertNullResult(result);
}
private void getTestNull(HTable ht, byte [] row, byte [] family,
private void getTestNull(Table ht, byte [] row, byte [] family,
byte [] value)
throws Exception {
@ -2668,12 +2668,12 @@ public class TestFromClientSide {
}
private void scanTestNull(HTable ht, byte[] row, byte[] family, byte[] value)
private void scanTestNull(Table ht, byte[] row, byte[] family, byte[] value)
throws Exception {
scanTestNull(ht, row, family, value, false);
}
private void scanTestNull(HTable ht, byte[] row, byte[] family, byte[] value,
private void scanTestNull(Table ht, byte[] row, byte[] family, byte[] value,
boolean isReversedScan) throws Exception {
Scan scan = new Scan();
@ -2701,7 +2701,7 @@ public class TestFromClientSide {
}
private void singleRowGetTest(HTable ht, byte [][] ROWS, byte [][] FAMILIES,
private void singleRowGetTest(Table ht, byte [][] ROWS, byte [][] FAMILIES,
byte [][] QUALIFIERS, byte [][] VALUES)
throws Exception {
@ -2799,7 +2799,7 @@ public class TestFromClientSide {
}
private void singleRowScanTest(HTable ht, byte [][] ROWS, byte [][] FAMILIES,
private void singleRowScanTest(Table ht, byte [][] ROWS, byte [][] FAMILIES,
byte [][] QUALIFIERS, byte [][] VALUES)
throws Exception {
@ -2901,7 +2901,7 @@ public class TestFromClientSide {
* Expects family and qualifier arrays to be valid for at least
* the range: idx-2 < idx < idx+2
*/
private void getVerifySingleColumn(HTable ht,
private void getVerifySingleColumn(Table ht,
byte [][] ROWS, int ROWIDX,
byte [][] FAMILIES, int FAMILYIDX,
byte [][] QUALIFIERS, int QUALIFIERIDX,
@ -2959,7 +2959,7 @@ public class TestFromClientSide {
* the range: idx-2 to idx+2
* Expects row array to be valid for at least idx to idx+2
*/
private void scanVerifySingleColumn(HTable ht,
private void scanVerifySingleColumn(Table ht,
byte [][] ROWS, int ROWIDX,
byte [][] FAMILIES, int FAMILYIDX,
byte [][] QUALIFIERS, int QUALIFIERIDX,
@ -3019,7 +3019,7 @@ public class TestFromClientSide {
* Verify we do not read any values by accident around a single column
* Same requirements as getVerifySingleColumn
*/
private void getVerifySingleEmpty(HTable ht,
private void getVerifySingleEmpty(Table ht,
byte [][] ROWS, int ROWIDX,
byte [][] FAMILIES, int FAMILYIDX,
byte [][] QUALIFIERS, int QUALIFIERIDX)
@ -3050,7 +3050,7 @@ public class TestFromClientSide {
}
private void scanVerifySingleEmpty(HTable ht,
private void scanVerifySingleEmpty(Table ht,
byte [][] ROWS, int ROWIDX,
byte [][] FAMILIES, int FAMILYIDX,
byte [][] QUALIFIERS, int QUALIFIERIDX)
@ -3275,7 +3275,7 @@ public class TestFromClientSide {
// Helpers
//
private Result getSingleScanResult(HTable ht, Scan scan) throws IOException {
private Result getSingleScanResult(Table ht, Scan scan) throws IOException {
ResultScanner scanner = ht.getScanner(scan);
Result result = scanner.next();
scanner.close();
@ -3335,7 +3335,7 @@ public class TestFromClientSide {
long [] STAMPS = makeStamps(20);
byte [][] VALUES = makeNAscii(VALUE, 20);
HTable ht = TEST_UTIL.createTable(TABLE, FAMILY, 10);
Table ht = TEST_UTIL.createTable(TABLE, FAMILY, 10);
// Insert 4 versions of same column
Put put = new Put(ROW);
@ -3546,7 +3546,7 @@ public class TestFromClientSide {
public void testUpdates() throws Exception {
byte [] TABLE = Bytes.toBytes("testUpdates");
HTable hTable = TEST_UTIL.createTable(TABLE, FAMILY, 10);
Table hTable = TEST_UTIL.createTable(TABLE, FAMILY, 10);
// Write a column with values at timestamp 1, 2 and 3
byte[] row = Bytes.toBytes("row1");
@ -3597,7 +3597,7 @@ public class TestFromClientSide {
String tableName = "testUpdatesWithMajorCompaction";
byte [] TABLE = Bytes.toBytes(tableName);
HTable hTable = TEST_UTIL.createTable(TABLE, FAMILY, 10);
Table hTable = TEST_UTIL.createTable(TABLE, FAMILY, 10);
HBaseAdmin admin = new HBaseAdmin(TEST_UTIL.getConfiguration());
// Write a column with values at timestamp 1, 2 and 3
@ -3659,7 +3659,7 @@ public class TestFromClientSide {
String tableName = "testMajorCompactionBetweenTwoUpdates";
byte [] TABLE = Bytes.toBytes(tableName);
HTable hTable = TEST_UTIL.createTable(TABLE, FAMILY, 10);
Table hTable = TEST_UTIL.createTable(TABLE, FAMILY, 10);
HBaseAdmin admin = new HBaseAdmin(TEST_UTIL.getConfiguration());
// Write a column with values at timestamp 1, 2 and 3
@ -3724,7 +3724,7 @@ public class TestFromClientSide {
@Test
public void testGet_EmptyTable() throws IOException {
HTable table = TEST_UTIL.createTable(Bytes.toBytes("testGet_EmptyTable"), FAMILY);
Table table = TEST_UTIL.createTable(Bytes.toBytes("testGet_EmptyTable"), FAMILY);
Get get = new Get(ROW);
get.addFamily(FAMILY);
Result r = table.get(get);
@ -3733,7 +3733,7 @@ public class TestFromClientSide {
@Test
public void testGet_NullQualifier() throws IOException {
HTable table = TEST_UTIL.createTable(Bytes.toBytes("testGet_NullQualifier"), FAMILY);
Table table = TEST_UTIL.createTable(Bytes.toBytes("testGet_NullQualifier"), FAMILY);
Put put = new Put(ROW);
put.add(FAMILY, QUALIFIER, VALUE);
table.put(put);
@ -3756,7 +3756,7 @@ public class TestFromClientSide {
@Test
public void testGet_NonExistentRow() throws IOException {
HTable table = TEST_UTIL.createTable(Bytes.toBytes("testGet_NonExistentRow"), FAMILY);
Table table = TEST_UTIL.createTable(Bytes.toBytes("testGet_NonExistentRow"), FAMILY);
Put put = new Put(ROW);
put.add(FAMILY, QUALIFIER, VALUE);
table.put(put);
@ -3783,7 +3783,7 @@ public class TestFromClientSide {
final byte [] row1 = Bytes.toBytes("row1");
final byte [] row2 = Bytes.toBytes("row2");
final byte [] value = Bytes.toBytes("abcd");
HTable table = TEST_UTIL.createTable(Bytes.toBytes("testPut"),
Table table = TEST_UTIL.createTable(Bytes.toBytes("testPut"),
new byte [][] {CONTENTS_FAMILY, SMALL_FAMILY});
Put put = new Put(row1);
put.add(CONTENTS_FAMILY, null, value);
@ -3820,7 +3820,7 @@ public class TestFromClientSide {
public void testPutNoCF() throws IOException {
final byte[] BAD_FAM = Bytes.toBytes("BAD_CF");
final byte[] VAL = Bytes.toBytes(100);
HTable table = TEST_UTIL.createTable(Bytes.toBytes("testPutNoCF"), new byte[][]{FAMILY});
Table table = TEST_UTIL.createTable(Bytes.toBytes("testPutNoCF"), new byte[][]{FAMILY});
boolean caughtNSCFE = false;
@ -3841,7 +3841,7 @@ public class TestFromClientSide {
final byte[] SMALL_FAMILY = Bytes.toBytes("smallfam");
final int NB_BATCH_ROWS = 10;
final byte[] value = Bytes.toBytes("abcd");
HTable table = TEST_UTIL.createTable(Bytes.toBytes("testRowsPut"),
Table table = TEST_UTIL.createTable(Bytes.toBytes("testRowsPut"),
new byte[][] {CONTENTS_FAMILY, SMALL_FAMILY });
ArrayList<Put> rowsUpdate = new ArrayList<Put>();
for (int i = 0; i < NB_BATCH_ROWS; i++) {
@ -3974,7 +3974,7 @@ public class TestFromClientSide {
final byte [] FAM1 = Bytes.toBytes("fam1");
final byte [] FAM2 = Bytes.toBytes("fam2");
// Open table
HTable table = TEST_UTIL.createTable(Bytes.toBytes("testHBase737"),
Table table = TEST_UTIL.createTable(Bytes.toBytes("testHBase737"),
new byte [][] {FAM1, FAM2});
// Insert some values
Put put = new Put(ROW);
@ -4069,7 +4069,7 @@ public class TestFromClientSide {
for (int i = 0; i < tables.length; i++) {
TEST_UTIL.createTable(tables[i], FAMILY);
}
HBaseAdmin admin = new HBaseAdmin(TEST_UTIL.getConfiguration());
Admin admin = new HBaseAdmin(TEST_UTIL.getConfiguration());
HTableDescriptor[] ts = admin.listTables();
HashSet<HTableDescriptor> result = new HashSet<HTableDescriptor>(ts.length);
for (int i = 0; i < ts.length; i++) {
@ -4156,14 +4156,14 @@ public class TestFromClientSide {
final byte[] attrValue = Bytes.toBytes("somevalue");
byte[] value = Bytes.toBytes("value");
HTable a = TEST_UTIL.createTable(tableAname, HConstants.CATALOG_FAMILY);
HTable b = TEST_UTIL.createTable(tableBname, HConstants.CATALOG_FAMILY);
Table a = TEST_UTIL.createTable(tableAname, HConstants.CATALOG_FAMILY);
Table b = TEST_UTIL.createTable(tableBname, HConstants.CATALOG_FAMILY);
Put put = new Put(ROW);
put.add(HConstants.CATALOG_FAMILY, null, value);
a.put(put);
// open a new connection to A and a connection to b
HTable newA = new HTable(TEST_UTIL.getConfiguration(), tableAname);
Table newA = new HTable(TEST_UTIL.getConfiguration(), tableAname);
// copy data from A to B
Scan scan = new Scan();
@ -4183,7 +4183,7 @@ public class TestFromClientSide {
}
// Opening a new connection to A will cause the tables to be reloaded
HTable anotherA = new HTable(TEST_UTIL.getConfiguration(), tableAname);
Table anotherA = new HTable(TEST_UTIL.getConfiguration(), tableAname);
Get get = new Get(ROW);
get.addFamily(HConstants.CATALOG_FAMILY);
anotherA.get(get);
@ -4345,7 +4345,7 @@ public class TestFromClientSide {
final byte [] TABLENAME = Bytes.toBytes("testMultiRowMutation");
final byte [] ROW1 = Bytes.toBytes("testRow1");
HTable t = TEST_UTIL.createTable(TABLENAME, FAMILY);
Table t = TEST_UTIL.createTable(TABLENAME, FAMILY);
Put p = new Put(ROW);
p.add(FAMILY, QUALIFIER, VALUE);
MutationProto m1 = ProtobufUtil.toMutation(MutationType.PUT, p);
@ -4374,7 +4374,7 @@ public class TestFromClientSide {
public void testRowMutation() throws Exception {
LOG.info("Starting testRowMutation");
final byte [] TABLENAME = Bytes.toBytes("testRowMutation");
HTable t = TEST_UTIL.createTable(TABLENAME, FAMILY);
Table t = TEST_UTIL.createTable(TABLENAME, FAMILY);
byte [][] QUALIFIERS = new byte [][] {
Bytes.toBytes("a"), Bytes.toBytes("b")
};
@ -4406,7 +4406,7 @@ public class TestFromClientSide {
public void testAppend() throws Exception {
LOG.info("Starting testAppend");
final byte [] TABLENAME = Bytes.toBytes("testAppend");
HTable t = TEST_UTIL.createTable(TABLENAME, FAMILY);
Table t = TEST_UTIL.createTable(TABLENAME, FAMILY);
byte[] v1 = Bytes.toBytes("42");
byte[] v2 = Bytes.toBytes("23");
byte [][] QUALIFIERS = new byte [][] {
@ -4436,7 +4436,7 @@ public class TestFromClientSide {
LOG.info("Starting testIncrementWithDeletes");
final TableName TABLENAME =
TableName.valueOf("testIncrementWithDeletes");
HTable ht = TEST_UTIL.createTable(TABLENAME, FAMILY);
Table ht = TEST_UTIL.createTable(TABLENAME, FAMILY);
final byte[] COLUMN = Bytes.toBytes("column");
ht.incrementColumnValue(ROW, FAMILY, COLUMN, 5);
@ -4457,7 +4457,7 @@ public class TestFromClientSide {
public void testIncrementingInvalidValue() throws Exception {
LOG.info("Starting testIncrementingInvalidValue");
final byte [] TABLENAME = Bytes.toBytes("testIncrementingInvalidValue");
HTable ht = TEST_UTIL.createTable(TABLENAME, FAMILY);
Table ht = TEST_UTIL.createTable(TABLENAME, FAMILY);
final byte[] COLUMN = Bytes.toBytes("column");
Put p = new Put(ROW);
// write an integer here (not a Long)
@ -4483,7 +4483,7 @@ public class TestFromClientSide {
public void testIncrementInvalidArguments() throws Exception {
LOG.info("Starting testIncrementInvalidArguments");
final byte[] TABLENAME = Bytes.toBytes("testIncrementInvalidArguments");
HTable ht = TEST_UTIL.createTable(TABLENAME, FAMILY);
Table ht = TEST_UTIL.createTable(TABLENAME, FAMILY);
final byte[] COLUMN = Bytes.toBytes("column");
try {
// try null row
@ -4538,7 +4538,7 @@ public class TestFromClientSide {
public void testIncrementOutOfOrder() throws Exception {
LOG.info("Starting testIncrementOutOfOrder");
final byte [] TABLENAME = Bytes.toBytes("testIncrementOutOfOrder");
HTable ht = TEST_UTIL.createTable(TABLENAME, FAMILY);
Table ht = TEST_UTIL.createTable(TABLENAME, FAMILY);
byte [][] QUALIFIERS = new byte [][] {
Bytes.toBytes("B"), Bytes.toBytes("A"), Bytes.toBytes("C")
@ -4578,7 +4578,7 @@ public class TestFromClientSide {
public void testIncrement() throws Exception {
LOG.info("Starting testIncrement");
final byte [] TABLENAME = Bytes.toBytes("testIncrement");
HTable ht = TEST_UTIL.createTable(TABLENAME, FAMILY);
Table ht = TEST_UTIL.createTable(TABLENAME, FAMILY);
byte [][] ROWS = new byte [][] {
Bytes.toBytes("a"), Bytes.toBytes("b"), Bytes.toBytes("c"),
@ -4656,7 +4656,7 @@ public class TestFromClientSide {
conf.set(HConstants.HBASE_CLIENT_IPC_POOL_TYPE, "round-robin");
conf.setInt(HConstants.HBASE_CLIENT_IPC_POOL_SIZE, poolSize);
HTable table = TEST_UTIL.createTable(tableName, new byte[][] { FAMILY },
Table table = TEST_UTIL.createTable(tableName, new byte[][] { FAMILY },
conf, Integer.MAX_VALUE);
final long ts = EnvironmentEdgeManager.currentTime();
@ -4693,7 +4693,7 @@ public class TestFromClientSide {
conf.set(HConstants.HBASE_CLIENT_IPC_POOL_TYPE, "thread-local");
conf.setInt(HConstants.HBASE_CLIENT_IPC_POOL_SIZE, poolSize);
final HTable table = TEST_UTIL.createTable(tableName,
final Table table = TEST_UTIL.createTable(tableName,
new byte[][] { FAMILY }, conf, 3);
final long ts = EnvironmentEdgeManager.currentTime();
@ -4771,7 +4771,7 @@ public class TestFromClientSide {
final byte [] anotherrow = Bytes.toBytes("anotherrow");
final byte [] value2 = Bytes.toBytes("abcd");
HTable table = TEST_UTIL.createTable(Bytes.toBytes("testCheckAndPut"),
Table table = TEST_UTIL.createTable(Bytes.toBytes("testCheckAndPut"),
new byte [][] {FAMILY});
Put put1 = new Put(ROW);
put1.add(FAMILY, QUALIFIER, VALUE);
@ -4813,7 +4813,7 @@ public class TestFromClientSide {
final byte [] value3 = Bytes.toBytes("cccc");
final byte [] value4 = Bytes.toBytes("dddd");
HTable table = TEST_UTIL.createTable(Bytes.toBytes("testCheckAndPutWithCompareOp"),
Table table = TEST_UTIL.createTable(Bytes.toBytes("testCheckAndPutWithCompareOp"),
new byte [][] {FAMILY});
Put put2 = new Put(ROW);
@ -4879,7 +4879,7 @@ public class TestFromClientSide {
final byte [] value3 = Bytes.toBytes("cccc");
final byte [] value4 = Bytes.toBytes("dddd");
HTable table = TEST_UTIL.createTable(Bytes.toBytes("testCheckAndDeleteWithCompareOp"),
Table table = TEST_UTIL.createTable(Bytes.toBytes("testCheckAndDeleteWithCompareOp"),
new byte [][] {FAMILY});
Put put2 = new Put(ROW);
@ -5176,7 +5176,7 @@ public class TestFromClientSide {
byte [] family1 = Bytes.toBytes("f1");
byte [] family2 = Bytes.toBytes("f2");
HTable table = TEST_UTIL.createTable(TABLE, new byte[][] {family1, family2}, 10);
HBaseAdmin admin = new HBaseAdmin(TEST_UTIL.getConfiguration());
Admin admin = new HBaseAdmin(TEST_UTIL.getConfiguration());
Map <HRegionInfo, ServerName> regionsMap = table.getRegionLocations();
assertEquals(1, regionsMap.size());
HRegionInfo regionInfo = regionsMap.keySet().iterator().next();
@ -5272,7 +5272,7 @@ public class TestFromClientSide {
@Test
public void testJira6912() throws Exception {
byte [] TABLE = Bytes.toBytes("testJira6912");
HTable foo = TEST_UTIL.createTable(TABLE, new byte[][] {FAMILY}, 10);
Table foo = TEST_UTIL.createTable(TABLE, new byte[][] {FAMILY}, 10);
List<Put> puts = new ArrayList<Put>();
for (int i=0;i !=100; i++){
@ -5297,7 +5297,7 @@ public class TestFromClientSide {
@Test
public void testScan_NullQualifier() throws IOException {
HTable table = TEST_UTIL.createTable(Bytes.toBytes("testScan_NullQualifier"), FAMILY);
Table table = TEST_UTIL.createTable(Bytes.toBytes("testScan_NullQualifier"), FAMILY);
Put put = new Put(ROW);
put.add(FAMILY, QUALIFIER, VALUE);
table.put(put);
@ -5326,7 +5326,7 @@ public class TestFromClientSide {
@Test
public void testNegativeTimestamp() throws IOException {
HTable table = TEST_UTIL.createTable(Bytes.toBytes("testNegativeTimestamp"), FAMILY);
Table table = TEST_UTIL.createTable(Bytes.toBytes("testNegativeTimestamp"), FAMILY);
try {
Put put = new Put(ROW, -1);
@ -5471,7 +5471,7 @@ public class TestFromClientSide {
@Test
public void testRawScanRespectsVersions() throws Exception {
byte[] TABLE = Bytes.toBytes("testRawScan");
HTable table = TEST_UTIL.createTable(TABLE, new byte[][] { FAMILY });
Table table = TEST_UTIL.createTable(TABLE, new byte[][] { FAMILY });
byte[] row = Bytes.toBytes("row");
// put the same row 4 times, with different values
@ -5547,7 +5547,7 @@ public class TestFromClientSide {
public void testSmallScan() throws Exception {
// Test Initialization.
byte[] TABLE = Bytes.toBytes("testSmallScan");
HTable table = TEST_UTIL.createTable(TABLE, FAMILY);
Table table = TEST_UTIL.createTable(TABLE, FAMILY);
// Insert one row each region
int insertNum = 10;
@ -5583,7 +5583,7 @@ public class TestFromClientSide {
@Test
public void testSuperSimpleWithReverseScan() throws Exception {
byte[] TABLE = Bytes.toBytes("testSuperSimpleWithReverseScan");
HTable ht = TEST_UTIL.createTable(TABLE, FAMILY);
Table ht = TEST_UTIL.createTable(TABLE, FAMILY);
Put put = new Put(Bytes.toBytes("0-b11111-0000000000000000000"));
put.add(FAMILY, QUALIFIER, VALUE);
ht.put(put);
@ -5629,7 +5629,7 @@ public class TestFromClientSide {
@Test
public void testFiltersWithReverseScan() throws Exception {
byte[] TABLE = Bytes.toBytes("testFiltersWithReverseScan");
HTable ht = TEST_UTIL.createTable(TABLE, FAMILY);
Table ht = TEST_UTIL.createTable(TABLE, FAMILY);
byte[][] ROWS = makeN(ROW, 10);
byte[][] QUALIFIERS = { Bytes.toBytes("col0-<d2v1>-<d3v2>"),
Bytes.toBytes("col1-<d2v1>-<d3v2>"),
@ -5669,7 +5669,7 @@ public class TestFromClientSide {
@Test
public void testKeyOnlyFilterWithReverseScan() throws Exception {
byte[] TABLE = Bytes.toBytes("testKeyOnlyFilterWithReverseScan");
HTable ht = TEST_UTIL.createTable(TABLE, FAMILY);
Table ht = TEST_UTIL.createTable(TABLE, FAMILY);
byte[][] ROWS = makeN(ROW, 10);
byte[][] QUALIFIERS = { Bytes.toBytes("col0-<d2v1>-<d3v2>"),
Bytes.toBytes("col1-<d2v1>-<d3v2>"),
@ -5710,7 +5710,7 @@ public class TestFromClientSide {
@Test
public void testSimpleMissingWithReverseScan() throws Exception {
byte[] TABLE = Bytes.toBytes("testSimpleMissingWithReverseScan");
HTable ht = TEST_UTIL.createTable(TABLE, FAMILY);
Table ht = TEST_UTIL.createTable(TABLE, FAMILY);
byte[][] ROWS = makeN(ROW, 4);
// Try to get a row on an empty table
@ -5775,7 +5775,7 @@ public class TestFromClientSide {
@Test
public void testNullWithReverseScan() throws Exception {
byte[] TABLE = Bytes.toBytes("testNullWithReverseScan");
HTable ht = TEST_UTIL.createTable(TABLE, FAMILY);
Table ht = TEST_UTIL.createTable(TABLE, FAMILY);
// Null qualifier (should work)
Put put = new Put(ROW);
put.add(FAMILY, null, VALUE);
@ -5816,7 +5816,7 @@ public class TestFromClientSide {
byte[][] FAMILIES = makeNAscii(FAMILY, 3);
byte[][] VALUES = makeN(VALUE, 5);
long[] ts = { 1000, 2000, 3000, 4000, 5000 };
HTable ht = TEST_UTIL.createTable(TABLE, FAMILIES,
Table ht = TEST_UTIL.createTable(TABLE, FAMILIES,
TEST_UTIL.getConfiguration(), 3);
Put put = new Put(ROW);
@ -6091,7 +6091,7 @@ public class TestFromClientSide {
table.close();
}
private void reverseScanTest(HTable table, boolean small) throws IOException {
private void reverseScanTest(Table table, boolean small) throws IOException {
// scan backward
Scan scan = new Scan();
scan.setReversed(true);

View File

@ -98,7 +98,7 @@ public class TestFromClientSide3 {
// Nothing to do.
}
private void randomCFPuts(HTable table, byte[] row, byte[] family, int nPuts)
private void randomCFPuts(Table table, byte[] row, byte[] family, int nPuts)
throws Exception {
Put put = new Put(row);
for (int i = 0; i < nPuts; i++) {
@ -269,7 +269,7 @@ public class TestFromClientSide3 {
@Test
public void testHTableBatchWithEmptyPut() throws Exception {
HTable table = TEST_UTIL.createTable(
Table table = TEST_UTIL.createTable(
Bytes.toBytes("testHTableBatchWithEmptyPut"), new byte[][] { FAMILY });
try {
List actions = (List) new ArrayList();
@ -296,7 +296,7 @@ public class TestFromClientSide3 {
// Test with a single region table.
HTable table = TEST_UTIL.createTable(
Table table = TEST_UTIL.createTable(
Bytes.toBytes("testHTableExistsMethodSingleRegionSingleGet"), new byte[][] { FAMILY });
Put put = new Put(ROW);
@ -336,7 +336,7 @@ public class TestFromClientSide3 {
@Test
public void testHTableExistsMethodMultipleRegionsSingleGet() throws Exception {
HTable table = TEST_UTIL.createTable(
Table table = TEST_UTIL.createTable(
Bytes.toBytes("testHTableExistsMethodMultipleRegionsSingleGet"), new byte[][] { FAMILY }, 1,
new byte[] { 0x00 }, new byte[] { (byte) 0xff }, 255);
Put put = new Put(ROW);
@ -409,7 +409,7 @@ public class TestFromClientSide3 {
HTableDescriptor desc = new HTableDescriptor(TableName.valueOf(Bytes.toBytes("test")));
desc.addFamily(new HColumnDescriptor(FAMILY));
admin.createTable(desc);
HTable table = new HTable(TEST_UTIL.getConfiguration(), "test");
Table table = new HTable(TEST_UTIL.getConfiguration(), "test");
Put put = new Put(ROW_BYTES);
put.add(FAMILY, COL_QUAL, VAL_BYTES);

View File

@ -36,7 +36,6 @@ import org.junit.Test;
import org.junit.experimental.categories.Category;
import org.mockito.Mockito;
import org.mortbay.log.Log;
import org.apache.hadoop.hbase.client.HConnectionTestingUtility;
import com.google.protobuf.RpcController;
import com.google.protobuf.ServiceException;
@ -71,7 +70,7 @@ public class TestHBaseAdminNoCluster {
thenThrow(new ServiceException("Test fail").initCause(new PleaseHoldException("test")));
Mockito.when(connection.getKeepAliveMasterService()).thenReturn(masterAdmin);
// Mock up our admin Interfaces
HBaseAdmin admin = new HBaseAdmin(configuration);
Admin admin = new HBaseAdmin(configuration);
try {
HTableDescriptor htd =
new HTableDescriptor(TableName.valueOf("testMasterMonitorCollableRetries"));

View File

@ -212,7 +212,7 @@ public class TestHCM {
@Test
public void testAdminFactory() throws IOException {
HConnection con1 = HConnectionManager.createConnection(TEST_UTIL.getConfiguration());
HBaseAdmin admin = (HBaseAdmin)con1.getAdmin();
Admin admin = con1.getAdmin();
assertTrue(admin.getConnection() == con1);
assertTrue(admin.getConfiguration() == TEST_UTIL.getConfiguration());
con1.close();
@ -425,7 +425,7 @@ public class TestHCM {
c2.setInt(HConstants.HBASE_CLIENT_RETRIES_NUMBER, 1); // Don't retry: retry = test failed
c2.setInt(RpcClient.IDLE_TIME, idleTime);
final HTable table = new HTable(c2, tableName.getBytes());
final Table table = new HTable(c2, tableName.getBytes());
Put put = new Put(ROW);
put.add(FAM_NAM, ROW, ROW);
@ -777,9 +777,9 @@ public class TestHCM {
*/
@Test
public void testConnectionManagement() throws Exception{
HTable table0 = TEST_UTIL.createTable(TABLE_NAME1, FAM_NAM);
Table table0 = TEST_UTIL.createTable(TABLE_NAME1, FAM_NAM);
HConnection conn = HConnectionManager.createConnection(TEST_UTIL.getConfiguration());
HTableInterface table = conn.getTable(TABLE_NAME1.getName());
Table table = conn.getTable(TABLE_NAME1.getName());
table.close();
assertFalse(conn.isClosed());
assertFalse(((HTable)table).getPool().isShutdown());
@ -1233,7 +1233,7 @@ public class TestHCM {
try {
c1 = ConnectionManager.getConnectionInternal(config);
LOG.info("HTable connection " + i + " " + c1);
HTable table = new HTable(config, TABLE_NAME4, pool);
Table table = new HTable(config, TABLE_NAME4, pool);
table.close();
LOG.info("HTable connection " + i + " closed " + c1);
} catch (Exception e) {
@ -1273,7 +1273,7 @@ public class TestHCM {
TEST_UTIL.createTable(tableName.getName(), new byte[][] {FAM_NAM}, config).close();
HConnection connection = HConnectionManager.createConnection(config);
HTableInterface table = connection.getTable(tableName);
Table table = connection.getTable(tableName);
// this will cache the meta location and table's region location
table.get(new Get(Bytes.toBytes("foo")));

View File

@ -147,7 +147,7 @@ public class TestMultiParallel {
*/
@Test(timeout=300000)
public void testActiveThreadsCount() throws Exception{
HTable table = new HTable(UTIL.getConfiguration(), TEST_TABLE);
Table table = new HTable(UTIL.getConfiguration(), TEST_TABLE);
List<Row> puts = constructPutRequests(); // creates a Put for every region
table.batch(puts);
Field poolField = table.getClass().getDeclaredField("pool");
@ -160,7 +160,7 @@ public class TestMultiParallel {
@Test(timeout=300000)
public void testBatchWithGet() throws Exception {
LOG.info("test=testBatchWithGet");
HTable table = new HTable(UTIL.getConfiguration(), TEST_TABLE);
Table table = new HTable(UTIL.getConfiguration(), TEST_TABLE);
// load test data
List<Row> puts = constructPutRequests();
@ -199,7 +199,7 @@ public class TestMultiParallel {
@Test
public void testBadFam() throws Exception {
LOG.info("test=testBadFam");
HTable table = new HTable(UTIL.getConfiguration(), TEST_TABLE);
Table table = new HTable(UTIL.getConfiguration(), TEST_TABLE);
List<Row> actions = new ArrayList<Row>();
Put p = new Put(Bytes.toBytes("row1"));
@ -321,7 +321,7 @@ public class TestMultiParallel {
@Test (timeout=300000)
public void testBatchWithPut() throws Exception {
LOG.info("test=testBatchWithPut");
HTable table = new HTable(UTIL.getConfiguration(), TEST_TABLE);
Table table = new HTable(UTIL.getConfiguration(), TEST_TABLE);
// put multiple rows using a batch
List<Row> puts = constructPutRequests();
@ -354,7 +354,7 @@ public class TestMultiParallel {
@Test(timeout=300000)
public void testBatchWithDelete() throws Exception {
LOG.info("test=testBatchWithDelete");
HTable table = new HTable(UTIL.getConfiguration(), TEST_TABLE);
Table table = new HTable(UTIL.getConfiguration(), TEST_TABLE);
// Load some data
List<Row> puts = constructPutRequests();
@ -383,7 +383,7 @@ public class TestMultiParallel {
@Test(timeout=300000)
public void testHTableDeleteWithList() throws Exception {
LOG.info("test=testHTableDeleteWithList");
HTable table = new HTable(UTIL.getConfiguration(), TEST_TABLE);
Table table = new HTable(UTIL.getConfiguration(), TEST_TABLE);
// Load some data
List<Row> puts = constructPutRequests();
@ -412,7 +412,7 @@ public class TestMultiParallel {
@Test(timeout=300000)
public void testBatchWithManyColsInOneRowGetAndPut() throws Exception {
LOG.info("test=testBatchWithManyColsInOneRowGetAndPut");
HTable table = new HTable(UTIL.getConfiguration(), TEST_TABLE);
Table table = new HTable(UTIL.getConfiguration(), TEST_TABLE);
List<Row> puts = new ArrayList<Row>();
for (int i = 0; i < 100; i++) {
@ -453,7 +453,7 @@ public class TestMultiParallel {
final byte[] QUAL2 = Bytes.toBytes("qual2");
final byte[] QUAL3 = Bytes.toBytes("qual3");
final byte[] QUAL4 = Bytes.toBytes("qual4");
HTable table = new HTable(UTIL.getConfiguration(), TEST_TABLE);
Table table = new HTable(UTIL.getConfiguration(), TEST_TABLE);
Delete d = new Delete(ONE_ROW);
table.delete(d);
Put put = new Put(ONE_ROW);
@ -531,7 +531,7 @@ public class TestMultiParallel {
Runnable r = new Runnable() {
@Override
public void run() {
HTable table = null;
Table table = null;
try {
table = new HTable(UTIL.getConfiguration(), TEST_TABLE);
} catch (IOException e) {
@ -573,7 +573,7 @@ public class TestMultiParallel {
@Test(timeout=300000)
public void testBatchWithMixedActions() throws Exception {
LOG.info("test=testBatchWithMixedActions");
HTable table = new HTable(UTIL.getConfiguration(), TEST_TABLE);
Table table = new HTable(UTIL.getConfiguration(), TEST_TABLE);
// Load some data to start
Object[] results = table.batch(constructPutRequests());
@ -666,7 +666,7 @@ public class TestMultiParallel {
return puts;
}
private void validateLoadedData(HTable table) throws IOException {
private void validateLoadedData(Table table) throws IOException {
// get the data back and validate that it is correct
for (byte[] k : KEYS) {
Get get = new Get(k);

View File

@ -87,7 +87,7 @@ public class TestMultipleTimestamps {
byte [][] FAMILIES = new byte[][] { FAMILY };
// create table; set versions to max...
HTable ht = TEST_UTIL.createTable(TABLE, FAMILIES, Integer.MAX_VALUE);
Table ht = TEST_UTIL.createTable(TABLE, FAMILIES, Integer.MAX_VALUE);
Integer[] putRows = new Integer[] {1, 3, 5, 7};
Integer[] putColumns = new Integer[] { 1, 3, 5};
@ -129,7 +129,7 @@ public class TestMultipleTimestamps {
byte [][] FAMILIES = new byte[][] { FAMILY };
// create table; set versions to max...
HTable ht = TEST_UTIL.createTable(TABLE, FAMILIES, Integer.MAX_VALUE);
Table ht = TEST_UTIL.createTable(TABLE, FAMILIES, Integer.MAX_VALUE);
Integer[] putRows = new Integer[] {1, 3, 5, 7};
Integer[] putColumns = new Integer[] { 1, 3, 5};
@ -170,7 +170,7 @@ public class TestMultipleTimestamps {
byte [][] FAMILIES = new byte[][] { FAMILY };
// create table; set versions to max...
HTable ht = TEST_UTIL.createTable(TABLE, FAMILIES, Integer.MAX_VALUE);
Table ht = TEST_UTIL.createTable(TABLE, FAMILIES, Integer.MAX_VALUE);
Integer[] putRows = new Integer[] {1, 3, 5, 7};
Integer[] putColumns = new Integer[] { 1, 3, 5};
@ -223,7 +223,7 @@ public class TestMultipleTimestamps {
byte [][] FAMILIES = new byte[][] { FAMILY };
// create table; set versions to max...
HTable ht = TEST_UTIL.createTable(TABLE, FAMILIES, Integer.MAX_VALUE);
Table ht = TEST_UTIL.createTable(TABLE, FAMILIES, Integer.MAX_VALUE);
Integer[] putRows1 = new Integer[] {1, 2, 3};
Integer[] putColumns1 = new Integer[] { 2, 5, 6};
@ -293,7 +293,7 @@ public class TestMultipleTimestamps {
byte [][] FAMILIES = new byte[][] { FAMILY };
// create table; set versions to max...
HTable ht = TEST_UTIL.createTable(TABLE, FAMILIES, Integer.MAX_VALUE);
Table ht = TEST_UTIL.createTable(TABLE, FAMILIES, Integer.MAX_VALUE);
// For row:0, col:0: insert versions 1 through 5.
putNVersions(ht, FAMILY, 0, 0, 1, 5);
@ -327,7 +327,7 @@ public class TestMultipleTimestamps {
byte [][] FAMILIES = new byte[][] { FAMILY };
// create table; set versions to max...
HTable ht = TEST_UTIL.createTable(TABLE, FAMILIES, Integer.MAX_VALUE);
Table ht = TEST_UTIL.createTable(TABLE, FAMILIES, Integer.MAX_VALUE);
// For row:0, col:0: insert versions 1 through 5.
putNVersions(ht, FAMILY, 0, 0, 1, 5);
@ -353,7 +353,7 @@ public class TestMultipleTimestamps {
byte [][] FAMILIES = new byte[][] { FAMILY };
// create table; set versions to max...
HTable ht = TEST_UTIL.createTable(TABLE, FAMILIES, Integer.MAX_VALUE);
Table ht = TEST_UTIL.createTable(TABLE, FAMILIES, Integer.MAX_VALUE);
// For row:0, col:0: insert versions 1 through 5.
putNVersions(ht, FAMILY, 0, 0, 1, 5);
@ -379,7 +379,7 @@ public class TestMultipleTimestamps {
byte [][] FAMILIES = new byte[][] { FAMILY };
// create table; set versions to max...
HTable ht = TEST_UTIL.createTable(TABLE, FAMILIES, Integer.MAX_VALUE);
Table ht = TEST_UTIL.createTable(TABLE, FAMILIES, Integer.MAX_VALUE);
// For row:0, col:0: insert versions 1 through 5.
putNVersions(ht, FAMILY, 0, 0, 1, 5);
@ -428,7 +428,7 @@ public class TestMultipleTimestamps {
* versions for the row/column specified by rowIdx & colIdx.
*
*/
private Cell[] getNVersions(HTable ht, byte[] cf, int rowIdx,
private Cell[] getNVersions(Table ht, byte[] cf, int rowIdx,
int colIdx, List<Long> versions)
throws IOException {
byte row[] = Bytes.toBytes("row:" + rowIdx);
@ -442,7 +442,7 @@ public class TestMultipleTimestamps {
return result.rawCells();
}
private ResultScanner scan(HTable ht, byte[] cf,
private ResultScanner scan(Table ht, byte[] cf,
Integer[] rowIndexes, Integer[] columnIndexes,
Long[] versions, int maxVersions)
throws IOException {
@ -463,7 +463,7 @@ public class TestMultipleTimestamps {
return scanner;
}
private void put(HTable ht, byte[] cf, Integer[] rowIndexes,
private void put(Table ht, byte[] cf, Integer[] rowIndexes,
Integer[] columnIndexes, Long[] versions)
throws IOException {
for (int rowIdx: rowIndexes) {
@ -485,7 +485,7 @@ public class TestMultipleTimestamps {
* Insert in specific row/column versions with timestamps
* versionStart..versionEnd.
*/
private void putNVersions(HTable ht, byte[] cf, int rowIdx, int colIdx,
private void putNVersions(Table ht, byte[] cf, int rowIdx, int colIdx,
long versionStart, long versionEnd)
throws IOException {
byte row[] = Bytes.toBytes("row:" + rowIdx);
@ -504,7 +504,7 @@ public class TestMultipleTimestamps {
* For row/column specified by rowIdx/colIdx, delete the cell
* corresponding to the specified version.
*/
private void deleteOneVersion(HTable ht, byte[] cf, int rowIdx,
private void deleteOneVersion(Table ht, byte[] cf, int rowIdx,
int colIdx, long version)
throws IOException {
byte row[] = Bytes.toBytes("row:" + rowIdx);
@ -518,7 +518,7 @@ public class TestMultipleTimestamps {
* For row/column specified by rowIdx/colIdx, delete all cells
* preceeding the specified version.
*/
private void deleteAllVersionsBefore(HTable ht, byte[] cf, int rowIdx,
private void deleteAllVersionsBefore(Table ht, byte[] cf, int rowIdx,
int colIdx, long version)
throws IOException {
byte row[] = Bytes.toBytes("row:" + rowIdx);
@ -528,7 +528,7 @@ public class TestMultipleTimestamps {
ht.delete(del);
}
private void deleteColumn(HTable ht, byte[] cf, int rowIdx, int colIdx) throws IOException {
private void deleteColumn(Table ht, byte[] cf, int rowIdx, int colIdx) throws IOException {
byte row[] = Bytes.toBytes("row:" + rowIdx);
byte column[] = Bytes.toBytes("column:" + colIdx);
Delete del = new Delete(row);
@ -536,7 +536,7 @@ public class TestMultipleTimestamps {
ht.delete(del);
}
private void deleteFamily(HTable ht, byte[] cf, int rowIdx) throws IOException {
private void deleteFamily(Table ht, byte[] cf, int rowIdx) throws IOException {
byte row[] = Bytes.toBytes("row:" + rowIdx);
Delete del = new Delete(row);
del.deleteFamily(cf);

View File

@ -52,7 +52,7 @@ public class TestPutWithDelete {
final TableName tableName = TableName.valueOf("TestPutWithDelete");
final byte[] rowKey = Bytes.toBytes("12345");
final byte[] family = Bytes.toBytes("cf");
HTable table = TEST_UTIL.createTable(tableName, family);
Table table = TEST_UTIL.createTable(tableName, family);
TEST_UTIL.waitTableAvailable(tableName.getName(), 5000);
try {
// put one row

View File

@ -139,7 +139,7 @@ public class TestReplicaWithCluster {
HTableDescriptor hdt = HTU.createTableDescriptor("testCreateDeleteTable");
hdt.setRegionReplication(NB_SERVERS);
hdt.addCoprocessor(SlowMeCopro.class.getName());
HTable table = HTU.createTable(hdt, new byte[][]{f}, HTU.getConfiguration());
Table table = HTU.createTable(hdt, new byte[][]{f}, HTU.getConfiguration());
Put p = new Put(row);
p.add(f, row, row);
@ -171,7 +171,7 @@ public class TestReplicaWithCluster {
HTableDescriptor hdt = HTU.createTableDescriptor("testChangeTable");
hdt.setRegionReplication(NB_SERVERS);
hdt.addCoprocessor(SlowMeCopro.class.getName());
HTable table = HTU.createTable(hdt, new byte[][]{f}, HTU.getConfiguration());
Table table = HTU.createTable(hdt, new byte[][]{f}, HTU.getConfiguration());
// basic test: it should work.
Put p = new Put(row);
@ -213,7 +213,7 @@ public class TestReplicaWithCluster {
}
HTU.getHBaseCluster().stopMaster(0);
HBaseAdmin admin = new HBaseAdmin(HTU.getConfiguration());
Admin admin = new HBaseAdmin(HTU.getConfiguration());
nHdt =admin.getTableDescriptor(hdt.getTableName());
Assert.assertEquals("fams=" + Arrays.toString(nHdt.getColumnFamilies()),
bHdt.getColumnFamilies().length + 1, nHdt.getColumnFamilies().length);
@ -253,7 +253,7 @@ public class TestReplicaWithCluster {
Put p = new Put(row);
p.add(row, row, row);
final HTable table = new HTable(HTU.getConfiguration(), hdt.getTableName());
final Table table = new HTable(HTU.getConfiguration(), hdt.getTableName());
table.put(p);
HTU.getHBaseAdmin().flush(table.getName());
@ -277,7 +277,7 @@ public class TestReplicaWithCluster {
table.close();
LOG.info("stale get on the first cluster done. Now for the second.");
final HTable table2 = new HTable(HTU.getConfiguration(), hdt.getTableName());
final Table table2 = new HTable(HTU.getConfiguration(), hdt.getTableName());
Waiter.waitFor(HTU.getConfiguration(), 1000, new Waiter.Predicate<Exception>() {
@Override
public boolean evaluate() throws Exception {
@ -312,7 +312,7 @@ public class TestReplicaWithCluster {
HTableDescriptor hdt = HTU.createTableDescriptor("testBulkLoad");
hdt.setRegionReplication(NB_SERVERS);
hdt.addCoprocessor(SlowMeCopro.class.getName());
HTable table = HTU.createTable(hdt, new byte[][]{f}, HTU.getConfiguration());
Table table = HTU.createTable(hdt, new byte[][]{f}, HTU.getConfiguration());
// create hfiles to load.
LOG.debug("Creating test data");

View File

@ -178,7 +178,7 @@ public class TestReplicasClient {
TestRegionServerNoMaster.stopMasterAndAssignMeta(HTU);
Configuration c = new Configuration(HTU.getConfiguration());
c.setInt(HConstants.HBASE_CLIENT_RETRIES_NUMBER, 1);
HBaseAdmin ha = new HBaseAdmin(c);
Admin ha = new HBaseAdmin(c);
for (boolean masterRuns = true; masterRuns; ) {
Thread.sleep(100);
try {

View File

@ -130,7 +130,7 @@ 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);
HTable table = new HTable(conf, name);
Table table = new HTable(conf, name);
table.setAutoFlushTo(false);
byte[] row = Bytes.toBytes("row");
Put p = new Put(row);
@ -188,7 +188,7 @@ public class TestRpcControllerFactory {
table.close();
}
int doScan(HTable table, Scan scan, int expectedCount) throws IOException {
int doScan(Table table, Scan scan, int expectedCount) throws IOException {
ResultScanner results = table.getScanner(scan);
results.next();
results.close();

View File

@ -65,7 +65,7 @@ public class TestScannerTimeout {
c.setInt(HConstants.THREAD_WAKE_FREQUENCY, THREAD_WAKE_FREQUENCY);
// We need more than one region server for this test
TEST_UTIL.startMiniCluster(2);
HTable table = TEST_UTIL.createTable(TABLE_NAME, SOME_BYTES);
Table table = TEST_UTIL.createTable(TABLE_NAME, SOME_BYTES);
for (int i = 0; i < NB_ROWS; i++) {
Put put = new Put(Bytes.toBytes(i));
put.add(SOME_BYTES, SOME_BYTES, SOME_BYTES);
@ -99,7 +99,7 @@ public class TestScannerTimeout {
LOG.info("START ************ test2481");
Scan scan = new Scan();
scan.setCaching(1);
HTable table =
Table table =
new HTable(new Configuration(TEST_UTIL.getConfiguration()), TABLE_NAME);
ResultScanner r = table.getScanner(scan);
int count = 0;
@ -139,7 +139,7 @@ public class TestScannerTimeout {
// this new table
Configuration conf = new Configuration(TEST_UTIL.getConfiguration());
conf.setInt(HConstants.HBASE_CLIENT_SCANNER_TIMEOUT_PERIOD, SCANNER_TIMEOUT * 100);
HTable higherScanTimeoutTable = new HTable(conf, TABLE_NAME);
Table higherScanTimeoutTable = new HTable(conf, TABLE_NAME);
ResultScanner r = higherScanTimeoutTable.getScanner(scan);
// This takes way less than SCANNER_TIMEOUT*100
rs.abort("die!");
@ -173,7 +173,7 @@ public class TestScannerTimeout {
Configuration conf = new Configuration(TEST_UTIL.getConfiguration());
conf.setInt(
HConstants.HBASE_CLIENT_SCANNER_TIMEOUT_PERIOD, SCANNER_TIMEOUT*100);
HTable table = new HTable(conf, TABLE_NAME);
Table table = new HTable(conf, TABLE_NAME);
LOG.info("START ************ TEST3686A---22");
ResultScanner r = table.getScanner(scan);
@ -212,7 +212,7 @@ public class TestScannerTimeout {
// this new table
Configuration conf = new Configuration(TEST_UTIL.getConfiguration());
conf.setInt(HConstants.HBASE_CLIENT_SCANNER_TIMEOUT_PERIOD, SCANNER_TIMEOUT * 100);
HTable higherScanTimeoutTable = new HTable(conf, TABLE_NAME);
Table higherScanTimeoutTable = new HTable(conf, TABLE_NAME);
ResultScanner r = higherScanTimeoutTable.getScanner(scan);
int count = 1;
r.next();

View File

@ -103,7 +103,7 @@ public class TestScannersFromClientSide {
byte [] TABLE = Bytes.toBytes("testScanBatch");
byte [][] QUALIFIERS = HTestConst.makeNAscii(QUALIFIER, 8);
HTable ht = TEST_UTIL.createTable(TABLE, FAMILY);
Table ht = TEST_UTIL.createTable(TABLE, FAMILY);
Put put;
Scan scan;
@ -179,7 +179,7 @@ public class TestScannersFromClientSide {
byte [][] FAMILIES = HTestConst.makeNAscii(FAMILY, 3);
byte [][] QUALIFIERS = HTestConst.makeNAscii(QUALIFIER, 20);
HTable ht = TEST_UTIL.createTable(TABLE, FAMILIES);
Table ht = TEST_UTIL.createTable(TABLE, FAMILIES);
Get get;
Put put;
@ -300,7 +300,7 @@ public class TestScannersFromClientSide {
byte [][] FAMILIES = HTestConst.makeNAscii(FAMILY, 3);
byte [][] QUALIFIERS = HTestConst.makeNAscii(QUALIFIER, 10);
HTable ht = TEST_UTIL.createTable(TABLE, FAMILIES);
Table ht = TEST_UTIL.createTable(TABLE, FAMILIES);
Put put;
Scan scan;
@ -349,7 +349,7 @@ public class TestScannersFromClientSide {
byte [][] FAMILIES = HTestConst.makeNAscii(FAMILY, 3);
byte [][] QUALIFIERS = HTestConst.makeNAscii(QUALIFIER, 20);
HTable ht = TEST_UTIL.createTable(TABLE, FAMILIES);
Table ht = TEST_UTIL.createTable(TABLE, FAMILIES);
Get get;
Put put;

View File

@ -212,7 +212,7 @@ public class TestSnapshotCloneIndependence {
TableName cloneTableName = TableName.valueOf("test-clone-" + localTableName);
admin.cloneSnapshot(snapshotName, cloneTableName);
HTable clonedTable = new HTable(UTIL.getConfiguration(), cloneTableName);
Table clonedTable = new HTable(UTIL.getConfiguration(), cloneTableName);
try {
final int clonedTableRowCount = UTIL.countRows(clonedTable);

View File

@ -169,7 +169,7 @@ public class TestSnapshotMetadata {
assertTrue(htd.getConfiguration().size() > 0);
admin.createTable(htd);
HTable original = new HTable(UTIL.getConfiguration(), originalTableName);
Table original = new HTable(UTIL.getConfiguration(), 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);
HTable clonedTable = new HTable(UTIL.getConfiguration(), clonedTableName);
Table clonedTable = new HTable(UTIL.getConfiguration(), clonedTableName);
HTableDescriptor cloneHtd = admin.getTableDescriptor(clonedTableName);
assertEquals(
originalTableDescription.replace(originalTableName.getNameAsString(),clonedTableNameAsString),
@ -312,7 +312,7 @@ public class TestSnapshotMetadata {
admin.enableTable(originalTableName);
// verify that the descrption is reverted
HTable original = new HTable(UTIL.getConfiguration(), originalTableName);
Table original = new HTable(UTIL.getConfiguration(), originalTableName);
try {
assertTrue(originalTableDescriptor.equals(admin.getTableDescriptor(originalTableName)));
assertTrue(originalTableDescriptor.equals(original.getTableDescriptor()));

View File

@ -96,7 +96,7 @@ public class TestTimestampsFilter {
Cell kvs[];
// create table; set versions to max...
HTable ht = TEST_UTIL.createTable(TABLE, FAMILIES, Integer.MAX_VALUE);
Table ht = TEST_UTIL.createTable(TABLE, FAMILIES, Integer.MAX_VALUE);
for (int rowIdx = 0; rowIdx < 5; rowIdx++) {
for (int colIdx = 0; colIdx < 5; colIdx++) {
@ -171,7 +171,7 @@ public class TestTimestampsFilter {
byte [][] FAMILIES = new byte[][] { FAMILY };
// create table; set versions to max...
HTable ht = TEST_UTIL.createTable(TABLE, FAMILIES, Integer.MAX_VALUE);
Table ht = TEST_UTIL.createTable(TABLE, FAMILIES, Integer.MAX_VALUE);
Put p = new Put(Bytes.toBytes("row"));
p.add(FAMILY, Bytes.toBytes("column0"), 3, Bytes.toBytes("value0-3"));
@ -231,7 +231,7 @@ public class TestTimestampsFilter {
byte [][] FAMILIES = new byte[][] { FAMILY };
// create table; set versions to max...
HTable ht = TEST_UTIL.createTable(TABLE, FAMILIES, Integer.MAX_VALUE);
Table ht = TEST_UTIL.createTable(TABLE, FAMILIES, Integer.MAX_VALUE);
// For row:0, col:0: insert versions 1 through 5.
putNVersions(ht, FAMILY, 0, 0, 1, 5);
@ -254,7 +254,7 @@ public class TestTimestampsFilter {
ht.close();
}
private void verifyInsertedValues(HTable ht, byte[] cf) throws IOException {
private void verifyInsertedValues(Table ht, byte[] cf) throws IOException {
for (int rowIdx = 0; rowIdx < 5; rowIdx++) {
for (int colIdx = 0; colIdx < 5; colIdx++) {
// ask for versions that exist.
@ -313,7 +313,7 @@ public class TestTimestampsFilter {
* versions for the row/column specified by rowIdx & colIdx.
*
*/
private Cell[] getNVersions(HTable ht, byte[] cf, int rowIdx,
private Cell[] getNVersions(Table ht, byte[] cf, int rowIdx,
int colIdx, List<Long> versions)
throws IOException {
byte row[] = Bytes.toBytes("row:" + rowIdx);
@ -332,7 +332,7 @@ public class TestTimestampsFilter {
* Uses the TimestampFilter on a Scan to request a specified list of
* versions for the rows from startRowIdx to endRowIdx (both inclusive).
*/
private Result[] scanNVersions(HTable ht, byte[] cf, int startRowIdx,
private Result[] scanNVersions(Table ht, byte[] cf, int startRowIdx,
int endRowIdx, List<Long> versions)
throws IOException {
byte startRow[] = Bytes.toBytes("row:" + startRowIdx);
@ -349,7 +349,7 @@ public class TestTimestampsFilter {
* Insert in specific row/column versions with timestamps
* versionStart..versionEnd.
*/
private void putNVersions(HTable ht, byte[] cf, int rowIdx, int colIdx,
private void putNVersions(Table ht, byte[] cf, int rowIdx, int colIdx,
long versionStart, long versionEnd)
throws IOException {
byte row[] = Bytes.toBytes("row:" + rowIdx);
@ -368,7 +368,7 @@ public class TestTimestampsFilter {
* For row/column specified by rowIdx/colIdx, delete the cell
* corresponding to the specified version.
*/
private void deleteOneVersion(HTable ht, byte[] cf, int rowIdx,
private void deleteOneVersion(Table ht, byte[] cf, int rowIdx,
int colIdx, long version)
throws IOException {
byte row[] = Bytes.toBytes("row:" + rowIdx);

View File

@ -34,6 +34,7 @@ import org.apache.hadoop.hbase.TableName;
import org.apache.hadoop.hbase.client.HTable;
import org.apache.hadoop.hbase.client.Put;
import org.apache.hadoop.hbase.client.RetriesExhaustedWithDetailsException;
import org.apache.hadoop.hbase.client.Table;
import org.apache.hadoop.hbase.coprocessor.CoprocessorHost;
import org.apache.hadoop.hbase.util.Bytes;
import org.junit.After;
@ -80,7 +81,7 @@ public class TestConstraint {
Constraints.add(desc, CheckWasRunConstraint.class);
util.getHBaseAdmin().createTable(desc);
HTable table = new HTable(util.getConfiguration(), tableName);
Table table = new HTable(util.getConfiguration(), tableName);
try {
// test that we don't fail on a valid put
Put put = new Put(row1);
@ -112,7 +113,7 @@ public class TestConstraint {
Constraints.add(desc, AllFailConstraint.class);
util.getHBaseAdmin().createTable(desc);
HTable table = new HTable(util.getConfiguration(), tableName);
Table table = new HTable(util.getConfiguration(), tableName);
// test that we do fail on violation
Put put = new Put(row1);
@ -155,7 +156,7 @@ public class TestConstraint {
Constraints.disableConstraint(desc, AllFailConstraint.class);
util.getHBaseAdmin().createTable(desc);
HTable table = new HTable(util.getConfiguration(), tableName);
Table table = new HTable(util.getConfiguration(), tableName);
try {
// test that we don't fail because its disabled
Put put = new Put(row1);
@ -187,7 +188,7 @@ public class TestConstraint {
Constraints.disable(desc);
util.getHBaseAdmin().createTable(desc);
HTable table = new HTable(util.getConfiguration(), tableName);
Table table = new HTable(util.getConfiguration(), tableName);
try {
// test that we do fail on violation
Put put = new Put(row1);
@ -219,7 +220,7 @@ public class TestConstraint {
CheckWasRunConstraint.wasRun = false;
util.getHBaseAdmin().createTable(desc);
HTable table = new HTable(util.getConfiguration(), tableName);
Table table = new HTable(util.getConfiguration(), tableName);
// test that we do fail on violation
Put put = new Put(row1);

View File

@ -25,6 +25,8 @@ import java.util.Collections;
import java.util.Map;
import java.util.TreeMap;
import org.apache.hadoop.hbase.client.Admin;
import org.apache.hadoop.hbase.client.Table;
import org.apache.hadoop.hbase.util.ByteStringer;
import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
@ -83,14 +85,14 @@ public class TestBatchCoprocessorEndpoint {
conf.setStrings(CoprocessorHost.MASTER_COPROCESSOR_CONF_KEY,
ProtobufCoprocessorService.class.getName());
util.startMiniCluster(2);
HBaseAdmin admin = new HBaseAdmin(conf);
Admin admin = new HBaseAdmin(conf);
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();
HTable table = new HTable(conf, TEST_TABLE);
Table table = new HTable(conf, TEST_TABLE);
for (int i = 0; i < ROWSIZE; i++) {
Put put = new Put(ROWS[i]);
put.add(TEST_FAMILY, TEST_QUALIFIER, Bytes.toBytes(i));
@ -106,7 +108,7 @@ public class TestBatchCoprocessorEndpoint {
@Test
public void testAggregationNullResponse() throws Throwable {
HTable table = new HTable(util.getConfiguration(), TEST_TABLE);
Table table = new HTable(util.getConfiguration(), TEST_TABLE);
ColumnAggregationWithNullResponseProtos.SumRequest.Builder builder =
ColumnAggregationWithNullResponseProtos.SumRequest
.newBuilder();
@ -143,7 +145,7 @@ public class TestBatchCoprocessorEndpoint {
return ret;
}
private Map<byte[], SumResponse> sum(final HTable table, final byte[] family,
private Map<byte[], SumResponse> sum(final Table table, final byte[] family,
final byte[] qualifier, final byte[] start, final byte[] end) throws ServiceException,
Throwable {
ColumnAggregationProtos.SumRequest.Builder builder = ColumnAggregationProtos.SumRequest
@ -159,7 +161,7 @@ public class TestBatchCoprocessorEndpoint {
@Test
public void testAggregationWithReturnValue() throws Throwable {
HTable table = new HTable(util.getConfiguration(), TEST_TABLE);
Table table = new HTable(util.getConfiguration(), TEST_TABLE);
Map<byte[], SumResponse> results = sum(table, TEST_FAMILY, TEST_QUALIFIER, ROWS[0],
ROWS[ROWS.length - 1]);
int sumResult = 0;
@ -195,7 +197,7 @@ public class TestBatchCoprocessorEndpoint {
@Test
public void testAggregation() throws Throwable {
HTable table = new HTable(util.getConfiguration(), TEST_TABLE);
Table table = new HTable(util.getConfiguration(), TEST_TABLE);
Map<byte[], SumResponse> results = sum(table, TEST_FAMILY, TEST_QUALIFIER,
ROWS[0], ROWS[ROWS.length - 1]);
int sumResult = 0;
@ -228,7 +230,7 @@ public class TestBatchCoprocessorEndpoint {
@Test
public void testAggregationWithErrors() throws Throwable {
HTable table = new HTable(util.getConfiguration(), TEST_TABLE);
Table table = new HTable(util.getConfiguration(), TEST_TABLE);
final Map<byte[], ColumnAggregationWithErrorsProtos.SumResponse> results =
Collections.synchronizedMap(
new TreeMap<byte[], ColumnAggregationWithErrorsProtos.SumResponse>(

View File

@ -31,6 +31,7 @@ import java.util.NavigableMap;
import java.util.TreeMap;
import org.apache.hadoop.hbase.client.Admin;
import org.apache.hadoop.hbase.client.Table;
import org.apache.hadoop.hbase.util.ByteStringer;
import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
@ -92,14 +93,14 @@ public class TestCoprocessorEndpoint {
conf.setStrings(CoprocessorHost.MASTER_COPROCESSOR_CONF_KEY,
ProtobufCoprocessorService.class.getName());
util.startMiniCluster(2);
HBaseAdmin admin = new HBaseAdmin(conf);
Admin admin = new HBaseAdmin(conf);
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();
HTable table = new HTable(conf, TEST_TABLE);
Table table = new HTable(conf, TEST_TABLE);
for (int i = 0; i < ROWSIZE; i++) {
Put put = new Put(ROWS[i]);
put.add(TEST_FAMILY, TEST_QUALIFIER, Bytes.toBytes(i));
@ -113,7 +114,7 @@ public class TestCoprocessorEndpoint {
util.shutdownMiniCluster();
}
private Map<byte [], Long> sum(final HTable table, final byte [] family,
private Map<byte [], Long> sum(final Table table, final byte [] family,
final byte [] qualifier, final byte [] start, final byte [] end)
throws ServiceException, Throwable {
return table.coprocessorService(ColumnAggregationProtos.ColumnAggregationService.class,
@ -138,7 +139,7 @@ public class TestCoprocessorEndpoint {
@Test
public void testAggregation() throws Throwable {
HTable table = new HTable(util.getConfiguration(), TEST_TABLE);
Table table = new HTable(util.getConfiguration(), TEST_TABLE);
Map<byte[], Long> results = sum(table, TEST_FAMILY, TEST_QUALIFIER,
ROWS[0], ROWS[ROWS.length-1]);
int sumResult = 0;
@ -297,7 +298,7 @@ public class TestCoprocessorEndpoint {
Configuration configuration = new Configuration(util.getConfiguration());
// Make it not retry forever
configuration.setInt(HConstants.HBASE_CLIENT_RETRIES_NUMBER, 1);
HTable table = new HTable(configuration, TEST_TABLE);
Table table = new HTable(configuration, TEST_TABLE);
try {
CoprocessorRpcChannel protocol = table.coprocessorService(ROWS[0]);

View File

@ -33,7 +33,6 @@ import org.apache.hadoop.hbase.client.Append;
import org.apache.hadoop.hbase.client.Delete;
import org.apache.hadoop.hbase.client.Durability;
import org.apache.hadoop.hbase.client.Get;
import org.apache.hadoop.hbase.client.HTable;
import org.apache.hadoop.hbase.client.HTableInterface;
import org.apache.hadoop.hbase.client.Increment;
import org.apache.hadoop.hbase.client.Put;
@ -42,6 +41,7 @@ import org.apache.hadoop.hbase.client.ResultScanner;
import org.apache.hadoop.hbase.client.Row;
import org.apache.hadoop.hbase.client.RowMutations;
import org.apache.hadoop.hbase.client.Scan;
import org.apache.hadoop.hbase.client.Table;
import org.apache.hadoop.hbase.client.coprocessor.Batch;
import org.apache.hadoop.hbase.ipc.CoprocessorRpcChannel;
import org.apache.hadoop.hbase.master.MasterCoprocessorHost;
@ -86,7 +86,7 @@ public class TestHTableWrapper {
}
private HTableInterface hTableInterface;
private HTable table;
private Table table;
@BeforeClass
public static void setupBeforeClass() throws Exception {

View File

@ -35,13 +35,12 @@ import org.apache.hadoop.hbase.MediumTests;
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.HBaseAdmin;
import org.apache.hadoop.hbase.client.HTable;
import org.apache.hadoop.hbase.client.HTableInterface;
import org.apache.hadoop.hbase.client.Put;
import org.apache.hadoop.hbase.client.Result;
import org.apache.hadoop.hbase.client.ResultScanner;
import org.apache.hadoop.hbase.client.Scan;
import org.apache.hadoop.hbase.client.Table;
import org.apache.hadoop.hbase.regionserver.wal.WALEdit;
import org.apache.hadoop.hbase.util.Threads;
import org.junit.After;
@ -69,7 +68,7 @@ public class TestOpenTableInCoprocessor {
@Override
public void prePut(final ObserverContext<RegionCoprocessorEnvironment> e, final Put put,
final WALEdit edit, final Durability durability) throws IOException {
HTableInterface table = e.getEnvironment().getTable(otherTable);
Table table = e.getEnvironment().getTable(otherTable);
table.put(put);
table.flushCommits();
completed[0] = true;
@ -102,7 +101,7 @@ public class TestOpenTableInCoprocessor {
@Override
public void prePut(final ObserverContext<RegionCoprocessorEnvironment> e, final Put put,
final WALEdit edit, final Durability durability) throws IOException {
HTableInterface table = e.getEnvironment().getTable(otherTable, getPool());
Table table = e.getEnvironment().getTable(otherTable, getPool());
Put p = new Put(new byte[] { 'a' });
p.add(family, null, new byte[] { 'a' });
try {
@ -162,14 +161,14 @@ public class TestOpenTableInCoprocessor {
admin.createTable(primary);
admin.createTable(other);
HTable table = new HTable(UTIL.getConfiguration(), "primary");
Table table = new HTable(UTIL.getConfiguration(), "primary");
Put p = new Put(new byte[] { 'a' });
p.add(family, null, new byte[] { 'a' });
table.put(p);
table.flushCommits();
table.close();
HTable target = new HTable(UTIL.getConfiguration(), otherTable);
Table target = new HTable(UTIL.getConfiguration(), otherTable);
assertTrue("Didn't complete update to target table!", completeCheck[0]);
assertEquals("Didn't find inserted row", 1, getKeyValueCount(target));
target.close();
@ -181,7 +180,7 @@ public class TestOpenTableInCoprocessor {
* @return number of keyvalues over all rows in the table
* @throws IOException
*/
private int getKeyValueCount(HTable table) throws IOException {
private int getKeyValueCount(Table table) throws IOException {
Scan scan = new Scan();
scan.setMaxVersions(Integer.MAX_VALUE - 1);

View File

@ -29,17 +29,16 @@ import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.hbase.Cell;
import org.apache.hadoop.hbase.HBaseConfiguration;
import org.apache.hadoop.hbase.HBaseTestingUtility;
import org.apache.hadoop.hbase.KeyValue;
import org.apache.hadoop.hbase.MediumTests;
import org.apache.hadoop.hbase.TableName;
import org.apache.hadoop.hbase.client.Admin;
import org.apache.hadoop.hbase.client.Delete;
import org.apache.hadoop.hbase.client.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.Durability;
import org.apache.hadoop.hbase.client.Table;
import org.apache.hadoop.hbase.regionserver.wal.WALEdit;
import org.apache.hadoop.hbase.util.Bytes;
import org.apache.hadoop.hbase.util.EnvironmentEdgeManager;
@ -93,7 +92,7 @@ public class TestRegionObserverBypass {
*/
@Test
public void testSimple() throws Exception {
HTable t = new HTable(util.getConfiguration(), tableName);
Table t = new HTable(util.getConfiguration(), tableName);
Put p = new Put(row1);
p.add(test,dummy,dummy);
// before HBASE-4331, this would throw an exception
@ -112,7 +111,7 @@ public class TestRegionObserverBypass {
//previous deletes will eclipse successive puts having the same timestamp
EnvironmentEdgeManagerTestHelper.injectEdge(new IncrementingEnvironmentEdge());
HTable t = new HTable(util.getConfiguration(), tableName);
Table t = new HTable(util.getConfiguration(), tableName);
List<Put> puts = new ArrayList<Put>();
Put p = new Put(row1);
p.add(dummy,dummy,dummy);
@ -197,7 +196,7 @@ public class TestRegionObserverBypass {
EnvironmentEdgeManager.reset();
}
private void checkRowAndDelete(HTable t, byte[] row, int count) throws IOException {
private void checkRowAndDelete(Table t, byte[] row, int count) throws IOException {
Get g = new Get(row);
Result r = t.get(g);
assertEquals(count, r.size());

View File

@ -58,6 +58,7 @@ import org.apache.hadoop.hbase.client.Result;
import org.apache.hadoop.hbase.client.ResultScanner;
import org.apache.hadoop.hbase.client.RowMutations;
import org.apache.hadoop.hbase.client.Scan;
import org.apache.hadoop.hbase.client.Table;
import org.apache.hadoop.hbase.io.hfile.CacheConfig;
import org.apache.hadoop.hbase.io.hfile.HFile;
import org.apache.hadoop.hbase.io.hfile.HFileContext;
@ -114,7 +115,7 @@ public class TestRegionObserverInterface {
TableName tableName = TableName.valueOf(TEST_TABLE.getNameAsString() + ".testRegionObserver");
// recreate table every time in order to reset the status of the
// coprocessor.
HTable table = util.createTable(tableName, new byte[][] {A, B, C});
Table table = util.createTable(tableName, new byte[][] {A, B, C});
try {
verifyMethodResult(SimpleRegionObserver.class, new String[] { "hadPreGet", "hadPostGet",
"hadPrePut", "hadPostPut", "hadDelete", "hadPostStartRegionOperation",
@ -176,7 +177,7 @@ public class TestRegionObserverInterface {
@Test
public void testRowMutation() throws IOException {
TableName tableName = TableName.valueOf(TEST_TABLE.getNameAsString() + ".testRowMutation");
HTable table = util.createTable(tableName, new byte[][] {A, B, C});
Table table = util.createTable(tableName, new byte[][] {A, B, C});
try {
verifyMethodResult(SimpleRegionObserver.class,
new String[] {"hadPreGet", "hadPostGet", "hadPrePut", "hadPostPut",
@ -213,7 +214,7 @@ public class TestRegionObserverInterface {
@Test
public void testIncrementHook() throws IOException {
TableName tableName = TableName.valueOf(TEST_TABLE.getNameAsString() + ".testIncrementHook");
HTable table = util.createTable(tableName, new byte[][] {A, B, C});
Table table = util.createTable(tableName, new byte[][] {A, B, C});
try {
Increment inc = new Increment(Bytes.toBytes(0));
inc.addColumn(A, A, 1);
@ -241,7 +242,7 @@ public class TestRegionObserverInterface {
public void testCheckAndPutHooks() throws IOException {
TableName tableName =
TableName.valueOf(TEST_TABLE.getNameAsString() + ".testCheckAndPutHooks");
HTable table = util.createTable(tableName, new byte[][] {A, B, C});
Table table = util.createTable(tableName, new byte[][] {A, B, C});
try {
Put p = new Put(Bytes.toBytes(0));
p.add(A, A, A);
@ -272,7 +273,7 @@ public class TestRegionObserverInterface {
public void testCheckAndDeleteHooks() throws IOException {
TableName tableName =
TableName.valueOf(TEST_TABLE.getNameAsString() + ".testCheckAndDeleteHooks");
HTable table = util.createTable(tableName, new byte[][] {A, B, C});
Table table = util.createTable(tableName, new byte[][] {A, B, C});
try {
Put p = new Put(Bytes.toBytes(0));
p.add(A, A, A);
@ -302,7 +303,7 @@ public class TestRegionObserverInterface {
@Test
public void testAppendHook() throws IOException {
TableName tableName = TableName.valueOf(TEST_TABLE.getNameAsString() + ".testAppendHook");
HTable table = util.createTable(tableName, new byte[][] {A, B, C});
Table table = util.createTable(tableName, new byte[][] {A, B, C});
try {
Append app = new Append(Bytes.toBytes(0));
app.add(A, A, A);
@ -341,7 +342,7 @@ public class TestRegionObserverInterface {
new Boolean[] {false, false, false, false}
);
HTable table = new HTable(util.getConfiguration(), tableName);
Table table = new HTable(util.getConfiguration(), tableName);
Put put = new Put(ROW);
put.add(A, A, A);
table.put(put);
@ -391,7 +392,7 @@ public class TestRegionObserverInterface {
new Boolean[] {false, false}
);
HTable table = new HTable(util.getConfiguration(), tableName);
Table table = new HTable(util.getConfiguration(), tableName);
Put put = new Put(ROW);
put.add(A, A, A);
table.put(put);
@ -498,7 +499,7 @@ public class TestRegionObserverInterface {
htd.addCoprocessor(EvenOnlyCompactor.class.getName());
admin.createTable(htd);
HTable table = new HTable(util.getConfiguration(), compactTable);
Table table = new HTable(util.getConfiguration(), compactTable);
for (long i=1; i<=10; i++) {
byte[] iBytes = Bytes.toBytes(i);
Put put = new Put(iBytes);
@ -560,7 +561,7 @@ public class TestRegionObserverInterface {
String testName = TestRegionObserverInterface.class.getName()+".bulkLoadHFileTest";
TableName tableName = TableName.valueOf(TEST_TABLE.getNameAsString() + ".bulkLoadHFileTest");
Configuration conf = util.getConfiguration();
HTable table = util.createTable(tableName, new byte[][] {A, B, C});
Table table = util.createTable(tableName, new byte[][] {A, B, C});
try {
verifyMethodResult(SimpleRegionObserver.class,
new String[] {"hadPreBulkLoadHFile", "hadPostBulkLoadHFile"},

View File

@ -48,6 +48,7 @@ import org.apache.hadoop.hbase.client.IsolationLevel;
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.filter.FilterBase;
import org.apache.hadoop.hbase.regionserver.HRegion;
import org.apache.hadoop.hbase.regionserver.HRegionServer;
@ -257,7 +258,7 @@ public class TestRegionObserverScannerOpenHook {
Admin admin = UTIL.getHBaseAdmin();
admin.createTable(desc);
HTable table = new HTable(conf, desc.getTableName());
Table table = new HTable(conf, desc.getTableName());
// put a row and flush it to disk
Put put = new Put(ROW);

View File

@ -36,6 +36,7 @@ import org.apache.hadoop.hbase.MediumTests;
import org.apache.hadoop.hbase.MiniHBaseCluster;
import org.apache.hadoop.hbase.TableName;
import org.apache.hadoop.hbase.MetaTableAccessor;
import org.apache.hadoop.hbase.client.Admin;
import org.apache.hadoop.hbase.client.HBaseAdmin;
import org.apache.hadoop.hbase.client.Mutation;
import org.apache.hadoop.hbase.regionserver.HRegion;
@ -74,7 +75,7 @@ public class TestRegionServerObserver {
// Start the cluster
HBaseTestingUtility TEST_UTIL = new HBaseTestingUtility(conf);
TEST_UTIL.startMiniCluster(NUM_MASTERS, NUM_RS);
HBaseAdmin admin = new HBaseAdmin(conf);
Admin admin = new HBaseAdmin(conf);
try {
MiniHBaseCluster cluster = TEST_UTIL.getHBaseCluster();
HRegionServer regionServer = cluster.getRegionServer(0);

View File

@ -34,6 +34,7 @@ import java.util.concurrent.atomic.AtomicInteger;
import java.util.concurrent.atomic.AtomicLong;
import org.apache.hadoop.hbase.TableName;
import org.apache.hadoop.hbase.client.Table;
import org.apache.hadoop.hbase.util.ByteStringer;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.hbase.Cell;
@ -44,7 +45,6 @@ import org.apache.hadoop.hbase.KeyValue;
import org.apache.hadoop.hbase.MediumTests;
import org.apache.hadoop.hbase.client.Delete;
import org.apache.hadoop.hbase.client.Get;
import org.apache.hadoop.hbase.client.HTable;
import org.apache.hadoop.hbase.client.IsolationLevel;
import org.apache.hadoop.hbase.client.Mutation;
import org.apache.hadoop.hbase.client.Put;
@ -107,7 +107,7 @@ public class TestRowProcessorEndpoint {
private static volatile int expectedCounter = 0;
private static int rowSize, row2Size;
private volatile static HTable table = null;
private volatile static Table table = null;
private volatile static boolean swapped = false;
private volatile CountDownLatch startSignal;
private volatile CountDownLatch doneSignal;
@ -196,7 +196,7 @@ public class TestRowProcessorEndpoint {
}
}
private int incrementCounter(HTable table) throws Throwable {
private int incrementCounter(Table table) throws Throwable {
CoprocessorRpcChannel channel = table.coprocessorService(ROW);
RowProcessorEndpoint.IncrementCounterProcessor processor =
new RowProcessorEndpoint.IncrementCounterProcessor(ROW);
@ -259,7 +259,7 @@ public class TestRowProcessorEndpoint {
}
}
private void swapRows(HTable table) throws Throwable {
private void swapRows(Table table) throws Throwable {
CoprocessorRpcChannel channel = table.coprocessorService(ROW);
RowProcessorEndpoint.RowSwapProcessor processor =
new RowProcessorEndpoint.RowSwapProcessor(ROW, ROW2);

View File

@ -19,7 +19,6 @@ package org.apache.hadoop.hbase.filter;
import static org.junit.Assert.*;
import java.io.IOException;
import java.util.ArrayList;
import java.util.HashMap;
import java.util.HashSet;
@ -30,14 +29,12 @@ import java.util.Set;
import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
import org.apache.hadoop.hbase.*;
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.Durability;
import org.apache.hadoop.hbase.regionserver.HRegion;
import org.apache.hadoop.hbase.regionserver.InternalScanner;
import org.apache.hadoop.hbase.client.Table;
import org.apache.hadoop.hbase.util.Bytes;
import org.junit.Test;
import org.junit.After;
@ -161,7 +158,7 @@ public class TestColumnRangeFilter {
public void TestColumnRangeFilterClient() throws Exception {
String family = "Family";
String table = "TestColumnRangeFilterClient";
HTable ht = TEST_UTIL.createTable(Bytes.toBytes(table),
Table ht = TEST_UTIL.createTable(Bytes.toBytes(table),
Bytes.toBytes(family), Integer.MAX_VALUE);
List<String> rows = generateRandomWords(10, 8);

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