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

View File

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

View File

@ -19,7 +19,6 @@
package org.apache.hadoop.hbase.client; package org.apache.hadoop.hbase.client;
import org.apache.hadoop.classification.InterfaceAudience; import org.apache.hadoop.classification.InterfaceAudience;
import org.apache.hadoop.classification.InterfaceStability;
import org.apache.hadoop.hbase.HRegionLocation; import org.apache.hadoop.hbase.HRegionLocation;
import java.io.IOException; 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>>(); Map<String, List<Put>> putMap = new HashMap<String, List<Put>>();
for (Put put: puts) { for (Put put: puts) {
@ -118,7 +117,7 @@ public class HTableUtil {
return putMap; 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>>(); Map<String, List<Row>> rowMap = new HashMap<String, List<Row>>();
for (Row row: rows) { 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.Result;
import org.apache.hadoop.hbase.client.ResultScanner; import org.apache.hadoop.hbase.client.ResultScanner;
import org.apache.hadoop.hbase.client.Scan; import org.apache.hadoop.hbase.client.Scan;
import org.apache.hadoop.hbase.client.Table;
import org.apache.hadoop.hbase.coprocessor.ColumnInterpreter; import org.apache.hadoop.hbase.coprocessor.ColumnInterpreter;
import org.apache.hadoop.hbase.ipc.BlockingRpcCallback; import org.apache.hadoop.hbase.ipc.BlockingRpcCallback;
import org.apache.hadoop.hbase.ipc.ServerRpcController; 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( 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) final TableName tableName, final ColumnInterpreter<R, S, P, Q, T> ci, final Scan scan)
throws Throwable { throws Throwable {
HTable table = null; Table table = null;
try { try {
table = new HTable(conf, tableName); table = new HTable(conf, tableName);
return max(table, ci, scan); return max(table, ci, scan);
@ -125,7 +126,7 @@ public class AggregationClient {
* & propagated to it. * & propagated to it.
*/ */
public <R, S, P extends Message, Q extends Message, T extends Message> 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 Scan scan) throws Throwable {
final AggregateRequest requestArg = validateArgAndGetPB(scan, ci, false); final AggregateRequest requestArg = validateArgAndGetPB(scan, ci, false);
class MaxCallBack implements Batch.Callback<R> { 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( 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) final TableName tableName, final ColumnInterpreter<R, S, P, Q, T> ci, final Scan scan)
throws Throwable { throws Throwable {
HTable table = null; Table table = null;
try { try {
table = new HTable(conf, tableName); table = new HTable(conf, tableName);
return min(table, ci, scan); return min(table, ci, scan);
@ -218,7 +219,7 @@ public class AggregationClient {
* @throws Throwable * @throws Throwable
*/ */
public <R, S, P extends Message, Q extends Message, T extends Message> 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 Scan scan) throws Throwable {
final AggregateRequest requestArg = validateArgAndGetPB(scan, ci, false); final AggregateRequest requestArg = validateArgAndGetPB(scan, ci, false);
class MinCallBack implements Batch.Callback<R> { 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( 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) final TableName tableName, final ColumnInterpreter<R, S, P, Q, T> ci, final Scan scan)
throws Throwable { throws Throwable {
HTable table = null; Table table = null;
try { try {
table = new HTable(conf, tableName); table = new HTable(conf, tableName);
return rowCount(table, ci, scan); return rowCount(table, ci, scan);
@ -301,7 +302,7 @@ public class AggregationClient {
* @throws Throwable * @throws Throwable
*/ */
public <R, S, P extends Message, Q extends Message, T extends Message> 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 ColumnInterpreter<R, S, P, Q, T> ci, final Scan scan) throws Throwable {
final AggregateRequest requestArg = validateArgAndGetPB(scan, ci, true); final AggregateRequest requestArg = validateArgAndGetPB(scan, ci, true);
class RowNumCallback implements Batch.Callback<Long> { 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( 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) final TableName tableName, final ColumnInterpreter<R, S, P, Q, T> ci, final Scan scan)
throws Throwable { throws Throwable {
HTable table = null; Table table = null;
try { try {
table = new HTable(conf, tableName); table = new HTable(conf, tableName);
return sum(table, ci, scan); return sum(table, ci, scan);
@ -371,7 +372,7 @@ public class AggregationClient {
* @throws Throwable * @throws Throwable
*/ */
public <R, S, P extends Message, Q extends Message, T extends Message> 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 Scan scan) throws Throwable {
final AggregateRequest requestArg = validateArgAndGetPB(scan, ci, false); 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( 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) final TableName tableName, final ColumnInterpreter<R, S, P, Q, T> ci, final Scan scan)
throws Throwable { throws Throwable {
HTable table = null; Table table = null;
try { try {
table = new HTable(conf, tableName); table = new HTable(conf, tableName);
return getAvgArgs(table, ci, scan); return getAvgArgs(table, ci, scan);
@ -443,7 +444,7 @@ public class AggregationClient {
* @throws Throwable * @throws Throwable
*/ */
private <R, S, P extends Message, Q extends Message, T extends Message> 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 ColumnInterpreter<R, S, P, Q, T> ci, final Scan scan) throws Throwable {
final AggregateRequest requestArg = validateArgAndGetPB(scan, ci, false); final AggregateRequest requestArg = validateArgAndGetPB(scan, ci, false);
class AvgCallBack implements Batch.Callback<Pair<S, Long>> { class AvgCallBack implements Batch.Callback<Pair<S, Long>> {
@ -523,7 +524,7 @@ public class AggregationClient {
* @throws Throwable * @throws Throwable
*/ */
public <R, S, P extends Message, Q extends Message, T extends Message> double avg( 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); Pair<S, Long> p = getAvgArgs(table, ci, scan);
return ci.divideForAvg(p.getFirst(), p.getSecond()); return ci.divideForAvg(p.getFirst(), p.getSecond());
} }
@ -540,7 +541,7 @@ public class AggregationClient {
* @throws Throwable * @throws Throwable
*/ */
private <R, S, P extends Message, Q extends Message, T extends Message> 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 ColumnInterpreter<R, S, P, Q, T> ci, final Scan scan) throws Throwable {
final AggregateRequest requestArg = validateArgAndGetPB(scan, ci, false); final AggregateRequest requestArg = validateArgAndGetPB(scan, ci, false);
class StdCallback implements Batch.Callback<Pair<List<S>, Long>> { 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> public <R, S, P extends Message, Q extends Message, T extends Message>
double std(final TableName tableName, ColumnInterpreter<R, S, P, Q, T> ci, double std(final TableName tableName, ColumnInterpreter<R, S, P, Q, T> ci,
Scan scan) throws Throwable { Scan scan) throws Throwable {
HTable table = null; Table table = null;
try { try {
table = new HTable(conf, tableName); table = new HTable(conf, tableName);
return std(table, ci, scan); return std(table, ci, scan);
@ -638,7 +639,7 @@ public class AggregationClient {
* @throws Throwable * @throws Throwable
*/ */
public <R, S, P extends Message, Q extends Message, T extends Message> double std( 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); Pair<List<S>, Long> p = getStdArgs(table, ci, scan);
double res = 0d; double res = 0d;
double avg = ci.divideForAvg(p.getFirst().get(0), p.getSecond()); 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> private <R, S, P extends Message, Q extends Message, T extends Message>
Pair<NavigableMap<byte[], List<S>>, List<S>> 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 ColumnInterpreter<R, S, P, Q, T> ci, final Scan scan) throws Throwable {
final AggregateRequest requestArg = validateArgAndGetPB(scan, ci, false); final AggregateRequest requestArg = validateArgAndGetPB(scan, ci, false);
final NavigableMap<byte[], List<S>> map = 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> public <R, S, P extends Message, Q extends Message, T extends Message>
R median(final TableName tableName, ColumnInterpreter<R, S, P, Q, T> ci, R median(final TableName tableName, ColumnInterpreter<R, S, P, Q, T> ci,
Scan scan) throws Throwable { Scan scan) throws Throwable {
HTable table = null; Table table = null;
try { try {
table = new HTable(conf, tableName); table = new HTable(conf, tableName);
return median(table, ci, scan); return median(table, ci, scan);
@ -749,7 +750,7 @@ public class AggregationClient {
* @throws Throwable * @throws Throwable
*/ */
public <R, S, P extends Message, Q extends Message, T extends Message> 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 { Scan scan) throws Throwable {
Pair<NavigableMap<byte[], List<S>>, List<S>> p = getMedianArgs(table, ci, scan); Pair<NavigableMap<byte[], List<S>>, List<S>> p = getMedianArgs(table, ci, scan);
byte[] startRow = null; 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.EMPTY_START_ROW;
import static org.apache.hadoop.hbase.HConstants.LAST_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.hbase.util.ByteStringer;
import org.apache.hadoop.classification.InterfaceAudience; import org.apache.hadoop.classification.InterfaceAudience;
import org.apache.hadoop.fs.Path; import org.apache.hadoop.fs.Path;
import org.apache.hadoop.hbase.TableName; 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.BlockingRpcCallback;
import org.apache.hadoop.hbase.ipc.CoprocessorRpcChannel; import org.apache.hadoop.hbase.ipc.CoprocessorRpcChannel;
import org.apache.hadoop.hbase.ipc.ServerRpcController; import org.apache.hadoop.hbase.ipc.ServerRpcController;
@ -47,9 +47,9 @@ import java.util.List;
*/ */
@InterfaceAudience.Private @InterfaceAudience.Private
public class SecureBulkLoadClient { public class SecureBulkLoadClient {
private HTable table; private Table table;
public SecureBulkLoadClient(HTable table) { public SecureBulkLoadClient(Table table) {
this.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.NamespaceDescriptor;
import org.apache.hadoop.hbase.TableName; import org.apache.hadoop.hbase.TableName;
import org.apache.hadoop.hbase.ZooKeeperConnectionException; import org.apache.hadoop.hbase.ZooKeeperConnectionException;
import org.apache.hadoop.hbase.client.Admin;
import org.apache.hadoop.hbase.client.HBaseAdmin; import org.apache.hadoop.hbase.client.HBaseAdmin;
import org.apache.hadoop.hbase.client.HTable; import org.apache.hadoop.hbase.client.HTable;
import org.apache.hadoop.hbase.client.Table;
import org.apache.hadoop.hbase.client.coprocessor.Batch; import org.apache.hadoop.hbase.client.coprocessor.Batch;
import org.apache.hadoop.hbase.ipc.BlockingRpcCallback; import org.apache.hadoop.hbase.ipc.BlockingRpcCallback;
import org.apache.hadoop.hbase.ipc.CoprocessorRpcChannel; import org.apache.hadoop.hbase.ipc.CoprocessorRpcChannel;
@ -70,7 +72,7 @@ public class AccessControlClient {
public static GrantResponse grant(Configuration conf, final TableName tableName, public static GrantResponse grant(Configuration conf, final TableName tableName,
final String userName, final byte[] family, final byte[] qual, final String userName, final byte[] family, final byte[] qual,
final AccessControlProtos.Permission.Action... actions) throws Throwable { final AccessControlProtos.Permission.Action... actions) throws Throwable {
HTable ht = null; Table ht = null;
try { try {
TableName aclTableName = TableName aclTableName =
TableName.valueOf(NamespaceDescriptor.SYSTEM_NAMESPACE_NAME_STR, "acl"); TableName.valueOf(NamespaceDescriptor.SYSTEM_NAMESPACE_NAME_STR, "acl");
@ -150,7 +152,7 @@ public class AccessControlClient {
public static RevokeResponse revoke(Configuration conf, final String username, public static RevokeResponse revoke(Configuration conf, final String username,
final TableName tableName, final byte[] family, final byte[] qualifier, final TableName tableName, final byte[] family, final byte[] qualifier,
final AccessControlProtos.Permission.Action... actions) throws Throwable { final AccessControlProtos.Permission.Action... actions) throws Throwable {
HTable ht = null; Table ht = null;
try { try {
TableName aclTableName = TableName.valueOf(NamespaceDescriptor.SYSTEM_NAMESPACE_NAME_STR, TableName aclTableName = TableName.valueOf(NamespaceDescriptor.SYSTEM_NAMESPACE_NAME_STR,
"acl"); "acl");
@ -211,8 +213,8 @@ public class AccessControlClient {
public static List<UserPermission> getUserPermissions(Configuration conf, String tableRegex) public static List<UserPermission> getUserPermissions(Configuration conf, String tableRegex)
throws Throwable { throws Throwable {
List<UserPermission> permList = new ArrayList<UserPermission>(); List<UserPermission> permList = new ArrayList<UserPermission>();
HTable ht = null; Table ht = null;
HBaseAdmin ha = null; Admin ha = null;
try { try {
TableName aclTableName = TableName aclTableName =
TableName.valueOf(NamespaceDescriptor.SYSTEM_NAMESPACE_NAME_STR, "acl"); 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.io.IOException;
import java.util.Map; import java.util.Map;
import org.apache.hadoop.hbase.client.Table;
import org.apache.hadoop.hbase.util.ByteStringer; import org.apache.hadoop.hbase.util.ByteStringer;
import org.apache.hadoop.classification.InterfaceAudience; import org.apache.hadoop.classification.InterfaceAudience;
import org.apache.hadoop.classification.InterfaceStability; import org.apache.hadoop.classification.InterfaceStability;
@ -72,7 +73,7 @@ public class VisibilityClient {
*/ */
public static VisibilityLabelsResponse addLabels(Configuration conf, final String[] labels) public static VisibilityLabelsResponse addLabels(Configuration conf, final String[] labels)
throws Throwable { throws Throwable {
HTable ht = null; Table ht = null;
try { try {
ht = new HTable(conf, LABELS_TABLE_NAME.getName()); ht = new HTable(conf, LABELS_TABLE_NAME.getName());
Batch.Call<VisibilityLabelsService, VisibilityLabelsResponse> callable = Batch.Call<VisibilityLabelsService, VisibilityLabelsResponse> callable =
@ -126,7 +127,7 @@ public class VisibilityClient {
* @throws Throwable * @throws Throwable
*/ */
public static GetAuthsResponse getAuths(Configuration conf, final String user) throws Throwable { public static GetAuthsResponse getAuths(Configuration conf, final String user) throws Throwable {
HTable ht = null; Table ht = null;
try { try {
ht = new HTable(conf, LABELS_TABLE_NAME.getName()); ht = new HTable(conf, LABELS_TABLE_NAME.getName());
Batch.Call<VisibilityLabelsService, GetAuthsResponse> callable = Batch.Call<VisibilityLabelsService, GetAuthsResponse> callable =
@ -168,7 +169,7 @@ public class VisibilityClient {
private static VisibilityLabelsResponse setOrClearAuths(Configuration conf, final String[] auths, private static VisibilityLabelsResponse setOrClearAuths(Configuration conf, final String[] auths,
final String user, final boolean setOrClear) throws IOException, ServiceException, Throwable { final String user, final boolean setOrClear) throws IOException, ServiceException, Throwable {
HTable ht = null; Table ht = null;
try { try {
ht = new HTable(conf, LABELS_TABLE_NAME.getName()); ht = new HTable(conf, LABELS_TABLE_NAME.getName());
Batch.Call<VisibilityLabelsService, VisibilityLabelsResponse> callable = Batch.Call<VisibilityLabelsService, VisibilityLabelsResponse> callable =

View File

@ -150,7 +150,7 @@ public class TestClientNoCluster extends Configured implements Tool {
Configuration localConfig = HBaseConfiguration.create(this.conf); Configuration localConfig = HBaseConfiguration.create(this.conf);
// This override mocks up our exists/get call to throw a RegionServerStoppedException. // This override mocks up our exists/get call to throw a RegionServerStoppedException.
localConfig.set("hbase.client.connection.impl", RpcTimeoutConnection.class.getName()); localConfig.set("hbase.client.connection.impl", RpcTimeoutConnection.class.getName());
HTable table = new HTable(localConfig, TableName.META_TABLE_NAME); Table table = new HTable(localConfig, TableName.META_TABLE_NAME);
Throwable t = null; Throwable t = null;
LOG.info("Start"); LOG.info("Start");
try { 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 -- // and it has expired. Otherwise, if this functionality is broke, all retries will be run --
// all ten of them -- and we'll get the RetriesExhaustedException exception. // all ten of them -- and we'll get the RetriesExhaustedException exception.
localConfig.setInt(HConstants.HBASE_CLIENT_META_OPERATION_TIMEOUT, pause - 1); localConfig.setInt(HConstants.HBASE_CLIENT_META_OPERATION_TIMEOUT, pause - 1);
HTable table = new HTable(localConfig, TableName.META_TABLE_NAME); Table table = new HTable(localConfig, TableName.META_TABLE_NAME);
Throwable t = null; Throwable t = null;
try { try {
// An exists call turns into a get w/ a flag. // An exists call turns into a get w/ a flag.
@ -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 // Go against meta else we will try to find first region for the table on construction which
// means we'll have to do a bunch more mocking. Tests that go against meta only should be // means we'll have to do a bunch more mocking. Tests that go against meta only should be
// good for a bit of testing. // good for a bit of testing.
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); ResultScanner scanner = table.getScanner(HConstants.CATALOG_FAMILY);
try { try {
Result result = null; 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 // Go against meta else we will try to find first region for the table on construction which
// means we'll have to do a bunch more mocking. Tests that go against meta only should be // means we'll have to do a bunch more mocking. Tests that go against meta only should be
// good for a bit of testing. // good for a bit of testing.
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); ResultScanner scanner = table.getScanner(HConstants.CATALOG_FAMILY);
try { try {
Result result = null; Result result = null;
@ -700,7 +700,7 @@ public class TestClientNoCluster extends Configured implements Tool {
* @throws IOException * @throws IOException
*/ */
static void cycle(int id, final Configuration c, final HConnection sharedConnection) 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); table.setAutoFlushTo(false);
long namespaceSpan = c.getLong("hbase.test.namespace.span", 1000000); long namespaceSpan = c.getLong("hbase.test.namespace.span", 1000000);
long startTime = System.currentTimeMillis(); long startTime = System.currentTimeMillis();

View File

@ -99,7 +99,7 @@ public class TestSnapshotFromAdmin {
builder.build(), builder.build(), builder.build(), builder.setDone(true).build()); builder.build(), builder.build(), builder.build(), builder.setDone(true).build());
// setup the admin and run the test // setup the admin and run the test
HBaseAdmin admin = new HBaseAdmin(mockConnection); Admin admin = new HBaseAdmin(mockConnection);
String snapshot = "snapshot"; String snapshot = "snapshot";
TableName table = TableName.valueOf("table"); TableName table = TableName.valueOf("table");
// get start time // get start time
@ -122,7 +122,7 @@ public class TestSnapshotFromAdmin {
.mock(ConnectionManager.HConnectionImplementation.class); .mock(ConnectionManager.HConnectionImplementation.class);
Configuration conf = HBaseConfiguration.create(); Configuration conf = HBaseConfiguration.create();
Mockito.when(mockConnection.getConfiguration()).thenReturn(conf); Mockito.when(mockConnection.getConfiguration()).thenReturn(conf);
HBaseAdmin admin = new HBaseAdmin(mockConnection); Admin admin = new HBaseAdmin(mockConnection);
SnapshotDescription.Builder builder = SnapshotDescription.newBuilder(); SnapshotDescription.Builder builder = SnapshotDescription.newBuilder();
// check that invalid snapshot names fail // check that invalid snapshot names fail
failSnapshotStart(admin, builder.setName(HConstants.SNAPSHOT_DIR_NAME).build()); failSnapshotStart(admin, builder.setName(HConstants.SNAPSHOT_DIR_NAME).build());
@ -152,7 +152,7 @@ public class TestSnapshotFromAdmin {
admin.snapshot(builder.setName("snapshot").setTable("table").build()); 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 { try {
admin.snapshot(snapshot); admin.snapshot(snapshot);
fail("Snapshot should not have succeed with name:" + snapshot.getName()); 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.Put;
import org.apache.hadoop.hbase.client.Result; import org.apache.hadoop.hbase.client.Result;
import org.apache.hadoop.hbase.client.Scan; import org.apache.hadoop.hbase.client.Scan;
import org.apache.hadoop.hbase.client.Table;
import org.apache.hadoop.hbase.client.coprocessor.Batch; import org.apache.hadoop.hbase.client.coprocessor.Batch;
import org.apache.hadoop.hbase.coprocessor.CoprocessorHost; import org.apache.hadoop.hbase.coprocessor.CoprocessorHost;
import org.apache.hadoop.hbase.coprocessor.example.generated.BulkDeleteProtos.BulkDeleteRequest; import org.apache.hadoop.hbase.coprocessor.example.generated.BulkDeleteProtos.BulkDeleteRequest;
@ -78,7 +79,7 @@ public class TestBulkDeleteProtocol {
// @Ignore @Test // @Ignore @Test
public void testBulkDeleteEndpoint() throws Throwable { public void testBulkDeleteEndpoint() throws Throwable {
byte[] tableName = Bytes.toBytes("testBulkDeleteEndpoint"); byte[] tableName = Bytes.toBytes("testBulkDeleteEndpoint");
HTable ht = createTable(tableName); Table ht = createTable(tableName);
List<Put> puts = new ArrayList<Put>(100); List<Put> puts = new ArrayList<Put>(100);
for (int j = 0; j < 100; j++) { for (int j = 0; j < 100; j++) {
byte[] rowkey = Bytes.toBytes(j); byte[] rowkey = Bytes.toBytes(j);
@ -102,7 +103,7 @@ public class TestBulkDeleteProtocol {
throws Throwable { throws Throwable {
byte[] tableName = Bytes byte[] tableName = Bytes
.toBytes("testBulkDeleteEndpointWhenRowBatchSizeLessThanRowsToDeleteFromARegion"); .toBytes("testBulkDeleteEndpointWhenRowBatchSizeLessThanRowsToDeleteFromARegion");
HTable ht = createTable(tableName); Table ht = createTable(tableName);
List<Put> puts = new ArrayList<Put>(100); List<Put> puts = new ArrayList<Put>(100);
for (int j = 0; j < 100; j++) { for (int j = 0; j < 100; j++) {
byte[] rowkey = Bytes.toBytes(j); byte[] rowkey = Bytes.toBytes(j);
@ -123,7 +124,7 @@ public class TestBulkDeleteProtocol {
private long invokeBulkDeleteProtocol(byte[] tableName, final Scan scan, final int rowBatchSize, private long invokeBulkDeleteProtocol(byte[] tableName, final Scan scan, final int rowBatchSize,
final DeleteType deleteType, final Long timeStamp) throws Throwable { 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; long noOfDeletedRows = 0L;
Batch.Call<BulkDeleteService, BulkDeleteResponse> callable = Batch.Call<BulkDeleteService, BulkDeleteResponse> callable =
new Batch.Call<BulkDeleteService, BulkDeleteResponse>() { new Batch.Call<BulkDeleteService, BulkDeleteResponse>() {
@ -155,7 +156,7 @@ public class TestBulkDeleteProtocol {
// @Ignore @Test // @Ignore @Test
public void testBulkDeleteWithConditionBasedDelete() throws Throwable { public void testBulkDeleteWithConditionBasedDelete() throws Throwable {
byte[] tableName = Bytes.toBytes("testBulkDeleteWithConditionBasedDelete"); byte[] tableName = Bytes.toBytes("testBulkDeleteWithConditionBasedDelete");
HTable ht = createTable(tableName); Table ht = createTable(tableName);
List<Put> puts = new ArrayList<Put>(100); List<Put> puts = new ArrayList<Put>(100);
for (int j = 0; j < 100; j++) { for (int j = 0; j < 100; j++) {
byte[] rowkey = Bytes.toBytes(j); byte[] rowkey = Bytes.toBytes(j);
@ -185,7 +186,7 @@ public class TestBulkDeleteProtocol {
// @Ignore @Test // @Ignore @Test
public void testBulkDeleteColumn() throws Throwable { public void testBulkDeleteColumn() throws Throwable {
byte[] tableName = Bytes.toBytes("testBulkDeleteColumn"); byte[] tableName = Bytes.toBytes("testBulkDeleteColumn");
HTable ht = createTable(tableName); Table ht = createTable(tableName);
List<Put> puts = new ArrayList<Put>(100); List<Put> puts = new ArrayList<Put>(100);
for (int j = 0; j < 100; j++) { for (int j = 0; j < 100; j++) {
byte[] rowkey = Bytes.toBytes(j); byte[] rowkey = Bytes.toBytes(j);
@ -218,7 +219,7 @@ public class TestBulkDeleteProtocol {
htd.addFamily(new HColumnDescriptor(FAMILY1)); htd.addFamily(new HColumnDescriptor(FAMILY1));
htd.addFamily(new HColumnDescriptor(FAMILY2)); htd.addFamily(new HColumnDescriptor(FAMILY2));
TEST_UTIL.getHBaseAdmin().createTable(htd, Bytes.toBytes(0), Bytes.toBytes(120), 5); TEST_UTIL.getHBaseAdmin().createTable(htd, Bytes.toBytes(0), Bytes.toBytes(120), 5);
HTable ht = new HTable(TEST_UTIL.getConfiguration(), tableName); Table ht = new HTable(TEST_UTIL.getConfiguration(), tableName);
List<Put> puts = new ArrayList<Put>(100); List<Put> puts = new ArrayList<Put>(100);
for (int j = 0; j < 100; j++) { for (int j = 0; j < 100; j++) {
Put put = new Put(Bytes.toBytes(j)); Put put = new Put(Bytes.toBytes(j));
@ -245,7 +246,7 @@ public class TestBulkDeleteProtocol {
// @Ignore @Test // @Ignore @Test
public void testBulkDeleteColumnVersion() throws Throwable { public void testBulkDeleteColumnVersion() throws Throwable {
byte[] tableName = Bytes.toBytes("testBulkDeleteColumnVersion"); byte[] tableName = Bytes.toBytes("testBulkDeleteColumnVersion");
HTable ht = createTable(tableName); Table ht = createTable(tableName);
List<Put> puts = new ArrayList<Put>(100); List<Put> puts = new ArrayList<Put>(100);
for (int j = 0; j < 100; j++) { for (int j = 0; j < 100; j++) {
Put put = new Put(Bytes.toBytes(j)); Put put = new Put(Bytes.toBytes(j));
@ -293,7 +294,7 @@ public class TestBulkDeleteProtocol {
// @Ignore @Test // @Ignore @Test
public void testBulkDeleteColumnVersionBasedOnTS() throws Throwable { public void testBulkDeleteColumnVersionBasedOnTS() throws Throwable {
byte[] tableName = Bytes.toBytes("testBulkDeleteColumnVersionBasedOnTS"); byte[] tableName = Bytes.toBytes("testBulkDeleteColumnVersionBasedOnTS");
HTable ht = createTable(tableName); Table ht = createTable(tableName);
List<Put> puts = new ArrayList<Put>(100); List<Put> puts = new ArrayList<Put>(100);
for (int j = 0; j < 100; j++) { for (int j = 0; j < 100; j++) {
Put put = new Put(Bytes.toBytes(j)); Put put = new Put(Bytes.toBytes(j));
@ -340,7 +341,7 @@ public class TestBulkDeleteProtocol {
// @Ignore @Test // @Ignore @Test
public void testBulkDeleteWithNumberOfVersions() throws Throwable { public void testBulkDeleteWithNumberOfVersions() throws Throwable {
byte[] tableName = Bytes.toBytes("testBulkDeleteWithNumberOfVersions"); byte[] tableName = Bytes.toBytes("testBulkDeleteWithNumberOfVersions");
HTable ht = createTable(tableName); Table ht = createTable(tableName);
List<Put> puts = new ArrayList<Put>(100); List<Put> puts = new ArrayList<Put>(100);
for (int j = 0; j < 100; j++) { for (int j = 0; j < 100; j++) {
Put put = new Put(Bytes.toBytes(j)); Put put = new Put(Bytes.toBytes(j));
@ -422,13 +423,13 @@ public class TestBulkDeleteProtocol {
ht.close(); ht.close();
} }
private HTable createTable(byte[] tableName) throws IOException { private Table createTable(byte[] tableName) throws IOException {
HTableDescriptor htd = new HTableDescriptor(TableName.valueOf(tableName)); HTableDescriptor htd = new HTableDescriptor(TableName.valueOf(tableName));
HColumnDescriptor hcd = new HColumnDescriptor(FAMILY1); HColumnDescriptor hcd = new HColumnDescriptor(FAMILY1);
hcd.setMaxVersions(10);// Just setting 10 as I am not testing with more than 10 versions here hcd.setMaxVersions(10);// Just setting 10 as I am not testing with more than 10 versions here
htd.addFamily(hcd); htd.addFamily(hcd);
TEST_UTIL.getHBaseAdmin().createTable(htd, Bytes.toBytes(0), Bytes.toBytes(120), 5); TEST_UTIL.getHBaseAdmin().createTable(htd, Bytes.toBytes(0), Bytes.toBytes(120), 5);
HTable ht = new HTable(TEST_UTIL.getConfiguration(), tableName); Table ht = new HTable(TEST_UTIL.getConfiguration(), tableName);
return ht; 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.MediumTests;
import org.apache.hadoop.hbase.client.HTable; import org.apache.hadoop.hbase.client.HTable;
import org.apache.hadoop.hbase.client.Put; import org.apache.hadoop.hbase.client.Put;
import org.apache.hadoop.hbase.client.Table;
import org.apache.hadoop.hbase.client.coprocessor.Batch; import org.apache.hadoop.hbase.client.coprocessor.Batch;
import org.apache.hadoop.hbase.coprocessor.CoprocessorHost; import org.apache.hadoop.hbase.coprocessor.CoprocessorHost;
import org.apache.hadoop.hbase.coprocessor.example.generated.ExampleProtos; import org.apache.hadoop.hbase.coprocessor.example.generated.ExampleProtos;
import org.apache.hadoop.hbase.ipc.BlockingRpcCallback; import org.apache.hadoop.hbase.ipc.BlockingRpcCallback;
import org.apache.hadoop.hbase.ipc.ServerRpcController; import org.apache.hadoop.hbase.ipc.ServerRpcController;
import org.apache.hadoop.hbase.util.Bytes; 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 org.junit.experimental.categories.Category;
import java.io.IOException; import java.io.IOException;
@ -72,7 +69,7 @@ public class TestRowCountEndpoint {
// @Ignore @Test // @Ignore @Test
public void testEndpoint() throws Throwable { public void testEndpoint() throws Throwable {
HTable table = new HTable(CONF, TEST_TABLE); Table table = new HTable(CONF, TEST_TABLE);
// insert some test rows // insert some test rows
for (int i=0; i<5; i++) { for (int i=0; i<5; i++) {

View File

@ -32,6 +32,7 @@ import org.apache.hadoop.hbase.client.Get;
import org.apache.hadoop.hbase.client.HTable; import org.apache.hadoop.hbase.client.HTable;
import org.apache.hadoop.hbase.client.Put; import org.apache.hadoop.hbase.client.Put;
import org.apache.hadoop.hbase.client.Result; import org.apache.hadoop.hbase.client.Result;
import org.apache.hadoop.hbase.client.Table;
import org.apache.hadoop.hbase.coprocessor.CoprocessorHost; import org.apache.hadoop.hbase.coprocessor.CoprocessorHost;
import org.apache.hadoop.hbase.util.Bytes; import org.apache.hadoop.hbase.util.Bytes;
import org.apache.hadoop.hbase.util.EnvironmentEdgeManager; import org.apache.hadoop.hbase.util.EnvironmentEdgeManager;
@ -74,7 +75,7 @@ public class TestZooKeeperScanPolicyObserver {
.setTimeToLive(1); .setTimeToLive(1);
desc.addFamily(hcd); desc.addFamily(hcd);
TEST_UTIL.getHBaseAdmin().createTable(desc); 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(); long now = EnvironmentEdgeManager.currentTime();
ZooKeeperWatcher zkw = new ZooKeeperWatcher(TEST_UTIL.getConfiguration(), "test", null); 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.classification.InterfaceAudience;
import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.hbase.ClusterManager.ServiceType; 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.HBaseAdmin;
import org.apache.hadoop.hbase.client.HConnection; import org.apache.hadoop.hbase.client.HConnection;
import org.apache.hadoop.hbase.client.HConnectionManager; import org.apache.hadoop.hbase.client.HConnectionManager;
@ -45,7 +46,7 @@ import com.google.common.collect.Sets;
@InterfaceAudience.Private @InterfaceAudience.Private
public class DistributedHBaseCluster extends HBaseCluster { public class DistributedHBaseCluster extends HBaseCluster {
private HBaseAdmin admin; private Admin admin;
private ClusterManager clusterManager; 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.Result;
import org.apache.hadoop.hbase.client.ResultScanner; import org.apache.hadoop.hbase.client.ResultScanner;
import org.apache.hadoop.hbase.client.Scan; import org.apache.hadoop.hbase.client.Scan;
import org.apache.hadoop.hbase.client.Table;
import org.apache.hadoop.hbase.filter.CompareFilter; import org.apache.hadoop.hbase.filter.CompareFilter;
import org.apache.hadoop.hbase.filter.Filter; import org.apache.hadoop.hbase.filter.Filter;
import org.apache.hadoop.hbase.filter.SingleColumnValueFilter; import org.apache.hadoop.hbase.filter.SingleColumnValueFilter;
@ -222,7 +223,7 @@ public class IntegrationTestLazyCfLoading {
long maxRuntime = conf.getLong(timeoutKey, DEFAULT_TIMEOUT_MINUTES); long maxRuntime = conf.getLong(timeoutKey, DEFAULT_TIMEOUT_MINUTES);
long serverCount = util.getHBaseClusterInterface().getClusterStatus().getServersSize(); long serverCount = util.getHBaseClusterInterface().getClusterStatus().getServersSize();
long keysToWrite = serverCount * KEYS_TO_WRITE_PER_SERVER; long keysToWrite = serverCount * KEYS_TO_WRITE_PER_SERVER;
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 // Create multi-threaded writer and start it. We write multiple columns/CFs and verify
// their integrity, therefore multi-put is necessary. // their integrity, therefore multi-put is necessary.

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.HTable;
import org.apache.hadoop.hbase.client.Result; import org.apache.hadoop.hbase.client.Result;
import org.apache.hadoop.hbase.client.Scan; import org.apache.hadoop.hbase.client.Scan;
import org.apache.hadoop.hbase.client.Table;
import org.apache.hadoop.hbase.util.Bytes; import org.apache.hadoop.hbase.util.Bytes;
import org.apache.hadoop.io.LongWritable; import org.apache.hadoop.io.LongWritable;
import org.apache.hadoop.io.Text; import org.apache.hadoop.io.Text;
@ -153,7 +154,7 @@ public class IntegrationTestImportTsv implements Configurable, Tool {
assertEquals("Loading HFiles failed.", assertEquals("Loading HFiles failed.",
0, ToolRunner.run(new LoadIncrementalHFiles(new Configuration(getConf())), args)); 0, ToolRunner.run(new LoadIncrementalHFiles(new Configuration(getConf())), args));
HTable table = null; Table table = null;
Scan scan = new Scan() {{ Scan scan = new Scan() {{
setCacheBlocks(false); setCacheBlocks(false);
setCaching(1000); 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.RestartRsHoldingMetaAction;
import org.apache.hadoop.hbase.chaos.actions.RestartRsHoldingTableAction; import org.apache.hadoop.hbase.chaos.actions.RestartRsHoldingTableAction;
import org.apache.hadoop.hbase.chaos.factories.MonkeyConstants; 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.HBaseAdmin;
import org.apache.hadoop.hbase.client.HTable; import org.apache.hadoop.hbase.client.HTable;
import org.apache.hadoop.hbase.client.Put; import org.apache.hadoop.hbase.client.Put;
@ -56,6 +57,7 @@ import org.apache.hadoop.hbase.client.Result;
import org.apache.hadoop.hbase.client.ResultScanner; import org.apache.hadoop.hbase.client.ResultScanner;
import org.apache.hadoop.hbase.client.RetriesExhaustedException; import org.apache.hadoop.hbase.client.RetriesExhaustedException;
import org.apache.hadoop.hbase.client.Scan; import org.apache.hadoop.hbase.client.Scan;
import org.apache.hadoop.hbase.client.Table;
import org.apache.hadoop.hbase.coprocessor.CoprocessorException; import org.apache.hadoop.hbase.coprocessor.CoprocessorException;
import org.apache.hadoop.hbase.filter.KeyOnlyFilter; import org.apache.hadoop.hbase.filter.KeyOnlyFilter;
import org.apache.hadoop.hbase.ipc.FatalConnectionException; import org.apache.hadoop.hbase.ipc.FatalConnectionException;
@ -461,7 +463,7 @@ public class IntegrationTestMTTR {
*/ */
static class PutCallable extends TimingCallable { static class PutCallable extends TimingCallable {
private final HTable table; private final Table table;
public PutCallable(Future<?> f) throws IOException { public PutCallable(Future<?> f) throws IOException {
super(f); super(f);
@ -488,7 +490,7 @@ public class IntegrationTestMTTR {
* supplied future returns. Returns the max time taken to scan. * supplied future returns. Returns the max time taken to scan.
*/ */
static class ScanCallable extends TimingCallable { static class ScanCallable extends TimingCallable {
private final HTable table; private final Table table;
public ScanCallable(Future<?> f) throws IOException { public ScanCallable(Future<?> f) throws IOException {
super(f); super(f);
@ -531,7 +533,7 @@ public class IntegrationTestMTTR {
@Override @Override
protected boolean doAction() throws Exception { protected boolean doAction() throws Exception {
HBaseAdmin admin = null; Admin admin = null;
try { try {
admin = new HBaseAdmin(util.getConfiguration()); admin = new HBaseAdmin(util.getConfiguration());
ClusterStatus status = admin.getClusterStatus(); 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.IntegrationTests;
import org.apache.hadoop.hbase.MasterNotRunningException; import org.apache.hadoop.hbase.MasterNotRunningException;
import org.apache.hadoop.hbase.TableName; import org.apache.hadoop.hbase.TableName;
import org.apache.hadoop.hbase.client.Admin;
import org.apache.hadoop.hbase.client.Get; import org.apache.hadoop.hbase.client.Get;
import org.apache.hadoop.hbase.client.HBaseAdmin; import org.apache.hadoop.hbase.client.HBaseAdmin;
import org.apache.hadoop.hbase.client.HConnection; import org.apache.hadoop.hbase.client.HConnection;
@ -60,6 +61,7 @@ import org.apache.hadoop.hbase.client.Result;
import org.apache.hadoop.hbase.client.ResultScanner; import org.apache.hadoop.hbase.client.ResultScanner;
import org.apache.hadoop.hbase.client.Scan; import org.apache.hadoop.hbase.client.Scan;
import org.apache.hadoop.hbase.client.ScannerCallable; import org.apache.hadoop.hbase.client.ScannerCallable;
import org.apache.hadoop.hbase.client.Table;
import org.apache.hadoop.hbase.io.ImmutableBytesWritable; import org.apache.hadoop.hbase.io.ImmutableBytesWritable;
import org.apache.hadoop.hbase.mapreduce.TableMapReduceUtil; import org.apache.hadoop.hbase.mapreduce.TableMapReduceUtil;
import org.apache.hadoop.hbase.mapreduce.TableMapper; import org.apache.hadoop.hbase.mapreduce.TableMapper;
@ -448,7 +450,7 @@ public class IntegrationTestBigLinkedList extends IntegrationTestBase {
protected void createSchema() throws IOException { protected void createSchema() throws IOException {
Configuration conf = getConf(); Configuration conf = getConf();
HBaseAdmin admin = new HBaseAdmin(conf); Admin admin = new HBaseAdmin(conf);
TableName tableName = getTableName(conf); TableName tableName = getTableName(conf);
try { try {
if (!admin.tableExists(tableName)) { if (!admin.tableExists(tableName)) {
@ -873,7 +875,7 @@ public class IntegrationTestBigLinkedList extends IntegrationTestBase {
System.exit(-1); System.exit(-1);
} }
HTable table = new HTable(getConf(), getTableName(getConf())); Table table = new HTable(getConf(), getTableName(getConf()));
Scan scan = new Scan(); Scan scan = new Scan();
scan.setBatch(10000); scan.setBatch(10000);
@ -923,7 +925,7 @@ public class IntegrationTestBigLinkedList extends IntegrationTestBase {
org.apache.hadoop.hbase.client.Delete delete org.apache.hadoop.hbase.client.Delete delete
= new org.apache.hadoop.hbase.client.Delete(val); = new org.apache.hadoop.hbase.client.Delete(val);
HTable table = new HTable(getConf(), getTableName(getConf())); Table table = new HTable(getConf(), getTableName(getConf()));
table.delete(delete); table.delete(delete);
table.flushCommits(); table.flushCommits();
@ -969,7 +971,7 @@ public class IntegrationTestBigLinkedList extends IntegrationTestBase {
byte[] startKey = isSpecificStart ? Bytes.toBytesBinary(cmd.getOptionValue('s')) : null; byte[] startKey = isSpecificStart ? Bytes.toBytesBinary(cmd.getOptionValue('s')) : null;
int logEvery = cmd.hasOption('l') ? Integer.parseInt(cmd.getOptionValue('l')) : 1; int logEvery = cmd.hasOption('l') ? Integer.parseInt(cmd.getOptionValue('l')) : 1;
HTable table = new HTable(getConf(), getTableName(getConf())); Table table = new HTable(getConf(), getTableName(getConf()));
long numQueries = 0; long numQueries = 0;
// If isSpecificStart is set, only walk one list from that particular node. // If isSpecificStart is set, only walk one list from that particular node.
// Note that in case of circular (or P-shaped) list it will walk forever, as is // Note that in case of circular (or P-shaped) list it will walk forever, as is
@ -1005,7 +1007,7 @@ public class IntegrationTestBigLinkedList extends IntegrationTestBase {
return 0; 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 scan = new Scan();
scan.setStartRow(startKey); scan.setStartRow(startKey);
scan.setBatch(1); scan.setBatch(1);
@ -1028,7 +1030,7 @@ public class IntegrationTestBigLinkedList extends IntegrationTestBase {
return null; 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 get = new Get(row);
get.addColumn(FAMILY_NAME, COLUMN_PREV); get.addColumn(FAMILY_NAME, COLUMN_PREV);
Result result = table.get(get); 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.IntegrationTests;
import org.apache.hadoop.hbase.TableName; import org.apache.hadoop.hbase.TableName;
import org.apache.hadoop.hbase.chaos.factories.MonkeyFactory; import org.apache.hadoop.hbase.chaos.factories.MonkeyFactory;
import org.apache.hadoop.hbase.client.Admin;
import org.apache.hadoop.hbase.client.Delete; import org.apache.hadoop.hbase.client.Delete;
import org.apache.hadoop.hbase.client.HBaseAdmin; import org.apache.hadoop.hbase.client.HBaseAdmin;
import org.apache.hadoop.hbase.client.HConnection; import org.apache.hadoop.hbase.client.HConnection;
@ -46,6 +47,7 @@ import org.apache.hadoop.hbase.client.HTable;
import org.apache.hadoop.hbase.client.Put; import org.apache.hadoop.hbase.client.Put;
import org.apache.hadoop.hbase.client.Result; import org.apache.hadoop.hbase.client.Result;
import org.apache.hadoop.hbase.client.Scan; import org.apache.hadoop.hbase.client.Scan;
import org.apache.hadoop.hbase.client.Table;
import org.apache.hadoop.hbase.io.ImmutableBytesWritable; import org.apache.hadoop.hbase.io.ImmutableBytesWritable;
import org.apache.hadoop.hbase.io.hfile.HFile; import org.apache.hadoop.hbase.io.hfile.HFile;
import org.apache.hadoop.hbase.mapreduce.Import; import org.apache.hadoop.hbase.mapreduce.Import;
@ -130,7 +132,7 @@ public class IntegrationTestBigLinkedListWithVisibility extends IntegrationTestB
if(!acl) { if(!acl) {
LOG.info("No ACL available."); LOG.info("No ACL available.");
} }
HBaseAdmin admin = new HBaseAdmin(getConf()); Admin admin = new HBaseAdmin(getConf());
for (int i = 0; i < DEFAULT_TABLES_COUNT; i++) { for (int i = 0; i < DEFAULT_TABLES_COUNT; i++) {
TableName tableName = IntegrationTestBigLinkedListWithVisibility.getTableName(i); TableName tableName = IntegrationTestBigLinkedListWithVisibility.getTableName(i);
createTable(admin, tableName, false, acl); createTable(admin, tableName, false, acl);
@ -140,7 +142,7 @@ public class IntegrationTestBigLinkedListWithVisibility extends IntegrationTestB
admin.close(); admin.close();
} }
private void createTable(HBaseAdmin admin, TableName tableName, boolean setVersion, private void createTable(Admin admin, TableName tableName, boolean setVersion,
boolean acl) throws IOException { boolean acl) throws IOException {
if (!admin.tableExists(tableName)) { if (!admin.tableExists(tableName)) {
HTableDescriptor htd = new HTableDescriptor(tableName); HTableDescriptor htd = new HTableDescriptor(tableName);
@ -170,8 +172,8 @@ public class IntegrationTestBigLinkedListWithVisibility extends IntegrationTestB
} }
static class VisibilityGeneratorMapper extends GeneratorMapper { static class VisibilityGeneratorMapper extends GeneratorMapper {
HTable[] tables = new HTable[DEFAULT_TABLES_COUNT]; Table[] tables = new Table[DEFAULT_TABLES_COUNT];
HTable commonTable = null; Table commonTable = null;
@Override @Override
protected void setup(org.apache.hadoop.mapreduce.Mapper.Context context) throws IOException, 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); HTableDescriptor htd = new HTableDescriptor(table);
htd.addFamily(new HColumnDescriptor(TEST_FAMILY)); htd.addFamily(new HColumnDescriptor(TEST_FAMILY));
HBaseAdmin admin = new HBaseAdmin(getConf()); Admin admin = new HBaseAdmin(getConf());
if (doLoad) { if (doLoad) {
admin.createTable(htd, Bytes.toBytes(0L), Bytes.toBytes(-1L), numPresplits); admin.createTable(htd, Bytes.toBytes(0L), Bytes.toBytes(-1L), numPresplits);
doLoad(getConf(), htd); 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.Admin;
import org.apache.hadoop.hbase.client.Consistency; import org.apache.hadoop.hbase.client.Consistency;
import org.apache.hadoop.hbase.client.Get; 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.Result;
import org.apache.hadoop.hbase.client.Table;
import org.apache.hadoop.hbase.regionserver.StorefileRefresherChore; import org.apache.hadoop.hbase.regionserver.StorefileRefresherChore;
import org.apache.hadoop.hbase.util.LoadTestTool; import org.apache.hadoop.hbase.util.LoadTestTool;
import org.apache.hadoop.hbase.util.MultiThreadedReader; import org.apache.hadoop.hbase.util.MultiThreadedReader;
@ -326,7 +326,7 @@ public class IntegrationTestTimeBoundedRequestsWithRegionReplicas extends Integr
@Override @Override
protected void verifyResultsAndUpdateMetrics(boolean verify, Get[] gets, long elapsedNano, protected void verifyResultsAndUpdateMetrics(boolean verify, Get[] gets, long elapsedNano,
Result[] results, HTableInterface table, boolean isNullExpected) Result[] results, Table table, boolean isNullExpected)
throws IOException { throws IOException {
super.verifyResultsAndUpdateMetrics(verify, gets, elapsedNano, results, table, isNullExpected); super.verifyResultsAndUpdateMetrics(verify, gets, elapsedNano, results, table, isNullExpected);
for (Result r : results) { 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.HTableDescriptor;
import org.apache.hadoop.hbase.IntegrationTestingUtility; import org.apache.hadoop.hbase.IntegrationTestingUtility;
import org.apache.hadoop.hbase.IntegrationTests; 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.HBaseAdmin;
import org.apache.hadoop.hbase.client.Put; import org.apache.hadoop.hbase.client.Put;
import org.apache.hadoop.hbase.client.Result; import org.apache.hadoop.hbase.client.Result;
@ -369,7 +370,7 @@ public class IntegrationTestWithCellVisibilityLoadAndVerify extends IntegrationT
HTableDescriptor htd = new HTableDescriptor(getTablename()); HTableDescriptor htd = new HTableDescriptor(getTablename());
htd.addFamily(new HColumnDescriptor(TEST_FAMILY)); htd.addFamily(new HColumnDescriptor(TEST_FAMILY));
HBaseAdmin admin = new HBaseAdmin(getConf()); Admin admin = new HBaseAdmin(getConf());
try { try {
admin.createTable(htd, Bytes.toBytes(0L), Bytes.toBytes(-1L), numPresplits); admin.createTable(htd, Bytes.toBytes(0L), Bytes.toBytes(-1L), numPresplits);
} finally { } 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.Result;
import org.apache.hadoop.hbase.client.ResultScanner; import org.apache.hadoop.hbase.client.ResultScanner;
import org.apache.hadoop.hbase.client.Scan; import org.apache.hadoop.hbase.client.Scan;
import org.apache.hadoop.hbase.client.Table;
import org.apache.hadoop.hbase.util.AbstractHBaseTool; import org.apache.hadoop.hbase.util.AbstractHBaseTool;
import org.apache.hadoop.hbase.util.Bytes; import org.apache.hadoop.hbase.util.Bytes;
import org.apache.hadoop.util.ToolRunner; import org.apache.hadoop.util.ToolRunner;
@ -123,7 +124,7 @@ public class IntegrationTestSendTraceRequests extends AbstractHBaseTool {
ResultScanner rs = null; ResultScanner rs = null;
try { try {
innerScope = Trace.startSpan("Scan", Sampler.ALWAYS); innerScope = Trace.startSpan("Scan", Sampler.ALWAYS);
HTable ht = new HTable(util.getConfiguration(), tableName); Table ht = new HTable(util.getConfiguration(), tableName);
Scan s = new Scan(); Scan s = new Scan();
s.setStartRow(Bytes.toBytes(rowKeyQueue.take())); s.setStartRow(Bytes.toBytes(rowKeyQueue.take()));
s.setBatch(7); s.setBatch(7);
@ -171,7 +172,7 @@ public class IntegrationTestSendTraceRequests extends AbstractHBaseTool {
public void run() { public void run() {
HTable ht = null; Table ht = null;
try { try {
ht = new HTable(util.getConfiguration(), tableName); ht = new HTable(util.getConfiguration(), tableName);
} catch (IOException e) { } 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.InterfaceAudience;
import org.apache.hadoop.classification.InterfaceStability; import org.apache.hadoop.classification.InterfaceStability;
import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.hbase.client.Admin;
import org.apache.hadoop.hbase.client.HBaseAdmin; import org.apache.hadoop.hbase.client.HBaseAdmin;
import org.apache.hadoop.hbase.regionserver.HRegionServer; import org.apache.hadoop.hbase.regionserver.HRegionServer;
import org.apache.hadoop.hbase.security.User; import org.apache.hadoop.hbase.security.User;
@ -462,7 +463,7 @@ public class LocalHBaseCluster {
Configuration conf = HBaseConfiguration.create(); Configuration conf = HBaseConfiguration.create();
LocalHBaseCluster cluster = new LocalHBaseCluster(conf); LocalHBaseCluster cluster = new LocalHBaseCluster(conf);
cluster.startup(); cluster.startup();
HBaseAdmin admin = new HBaseAdmin(conf); Admin admin = new HBaseAdmin(conf);
HTableDescriptor htd = HTableDescriptor htd =
new HTableDescriptor(TableName.valueOf(cluster.getClass().getName())); new HTableDescriptor(TableName.valueOf(cluster.getClass().getName()));
admin.createTable(htd); 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.HTable;
import org.apache.hadoop.hbase.client.HTableInterface; import org.apache.hadoop.hbase.client.HTableInterface;
import org.apache.hadoop.hbase.client.HTableWrapper; import org.apache.hadoop.hbase.client.HTableWrapper;
import org.apache.hadoop.hbase.client.Table;
import org.apache.hadoop.hbase.util.Bytes; import org.apache.hadoop.hbase.util.Bytes;
import org.apache.hadoop.hbase.util.CoprocessorClassLoader; import org.apache.hadoop.hbase.util.CoprocessorClassLoader;
import org.apache.hadoop.hbase.util.SortedCopyOnWriteSet; import org.apache.hadoop.hbase.util.SortedCopyOnWriteSet;

View File

@ -26,6 +26,7 @@ import org.apache.hadoop.classification.InterfaceAudience;
import org.apache.hadoop.classification.InterfaceStability; import org.apache.hadoop.classification.InterfaceStability;
import org.apache.hadoop.hbase.HBaseConfiguration; import org.apache.hadoop.hbase.HBaseConfiguration;
import org.apache.hadoop.hbase.client.HTable; import org.apache.hadoop.hbase.client.HTable;
import org.apache.hadoop.hbase.client.RegionLocator;
import org.apache.hadoop.hbase.io.ImmutableBytesWritable; import org.apache.hadoop.hbase.io.ImmutableBytesWritable;
import org.apache.hadoop.hbase.util.Bytes; import org.apache.hadoop.hbase.util.Bytes;
import org.apache.hadoop.mapred.JobConf; import org.apache.hadoop.mapred.JobConf;
@ -46,7 +47,7 @@ import org.apache.hadoop.mapred.Partitioner;
public class HRegionPartitioner<K2,V2> public class HRegionPartitioner<K2,V2>
implements Partitioner<ImmutableBytesWritable, V2> { implements Partitioner<ImmutableBytesWritable, V2> {
private static final Log LOG = LogFactory.getLog(HRegionPartitioner.class); private static final Log LOG = LogFactory.getLog(HRegionPartitioner.class);
private HTable table; private RegionLocator table;
private byte[][] startKeys; private byte[][] startKeys;
public void configure(JobConf job) { 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.HBaseConfiguration;
import org.apache.hadoop.hbase.client.HTable; import org.apache.hadoop.hbase.client.HTable;
import org.apache.hadoop.hbase.client.Put; import org.apache.hadoop.hbase.client.Put;
import org.apache.hadoop.hbase.client.Table;
import org.apache.hadoop.hbase.io.ImmutableBytesWritable; import org.apache.hadoop.hbase.io.ImmutableBytesWritable;
import org.apache.hadoop.fs.FileAlreadyExistsException; import org.apache.hadoop.fs.FileAlreadyExistsException;
import org.apache.hadoop.mapred.InvalidJobConfException; import org.apache.hadoop.mapred.InvalidJobConfException;
@ -56,14 +57,14 @@ FileOutputFormat<ImmutableBytesWritable, Put> {
*/ */
protected static class TableRecordWriter protected static class TableRecordWriter
implements RecordWriter<ImmutableBytesWritable, Put> { implements RecordWriter<ImmutableBytesWritable, Put> {
private HTable m_table; private Table m_table;
/** /**
* Instantiate a TableRecordWriter with the HBase HClient for writing. * Instantiate a TableRecordWriter with the HBase HClient for writing.
* *
* @param table * @param table
*/ */
public TableRecordWriter(HTable table) { public TableRecordWriter(Table table) {
m_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.classification.InterfaceStability;
import org.apache.hadoop.hbase.client.HTable; import org.apache.hadoop.hbase.client.HTable;
import org.apache.hadoop.hbase.client.Result; import org.apache.hadoop.hbase.client.Result;
import org.apache.hadoop.hbase.client.Table;
import org.apache.hadoop.hbase.filter.Filter; import org.apache.hadoop.hbase.filter.Filter;
import org.apache.hadoop.hbase.io.ImmutableBytesWritable; import org.apache.hadoop.hbase.io.ImmutableBytesWritable;
import org.apache.hadoop.mapred.RecordReader; import org.apache.hadoop.mapred.RecordReader;
@ -62,7 +63,7 @@ implements RecordReader<ImmutableBytesWritable, Result> {
/** /**
* @param htable the {@link HTable} to scan. * @param htable the {@link HTable} to scan.
*/ */
public void setHTable(HTable htable) { public void setHTable(Table htable) {
this.recordReaderImpl.setHTable(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.ResultScanner;
import org.apache.hadoop.hbase.client.Scan; import org.apache.hadoop.hbase.client.Scan;
import org.apache.hadoop.hbase.client.ScannerCallable; import org.apache.hadoop.hbase.client.ScannerCallable;
import org.apache.hadoop.hbase.client.Table;
import org.apache.hadoop.hbase.filter.Filter; import org.apache.hadoop.hbase.filter.Filter;
import org.apache.hadoop.hbase.io.ImmutableBytesWritable; import org.apache.hadoop.hbase.io.ImmutableBytesWritable;
import org.apache.hadoop.hbase.mapreduce.TableInputFormat; import org.apache.hadoop.hbase.mapreduce.TableInputFormat;
@ -52,7 +53,7 @@ public class TableRecordReaderImpl {
private byte [] lastSuccessfulRow; private byte [] lastSuccessfulRow;
private Filter trrRowFilter; private Filter trrRowFilter;
private ResultScanner scanner; private ResultScanner scanner;
private HTable htable; private Table htable;
private byte [][] trrInputColumns; private byte [][] trrInputColumns;
private long timestamp; private long timestamp;
private int rowcount; private int rowcount;
@ -116,7 +117,7 @@ public class TableRecordReaderImpl {
/** /**
* @param htable the {@link HTable} to scan. * @param htable the {@link HTable} to scan.
*/ */
public void setHTable(HTable htable) { public void setHTable(Table htable) {
Configuration conf = htable.getConfiguration(); Configuration conf = htable.getConfiguration();
logScannerActivity = conf.getBoolean( logScannerActivity = conf.getBoolean(
ScannerCallable.LOG_SCANNER_ACTIVITY, false); 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.Result;
import org.apache.hadoop.hbase.client.ResultScanner; import org.apache.hadoop.hbase.client.ResultScanner;
import org.apache.hadoop.hbase.client.Scan; import org.apache.hadoop.hbase.client.Scan;
import org.apache.hadoop.hbase.client.Table;
import org.apache.hadoop.hbase.security.visibility.Authorizations; import org.apache.hadoop.hbase.security.visibility.Authorizations;
import org.apache.hadoop.hbase.security.visibility.VisibilityLabelOrdinalProvider; import org.apache.hadoop.hbase.security.visibility.VisibilityLabelOrdinalProvider;
import org.apache.hadoop.hbase.security.visibility.VisibilityUtils; import org.apache.hadoop.hbase.security.visibility.VisibilityUtils;
@ -66,7 +67,7 @@ public class DefaultVisibilityExpressionResolver implements VisibilityExpression
public void init() { public void init() {
// Reading all the labels and ordinal. // Reading all the labels and ordinal.
// This scan should be done by user with global_admin previliges.. Ensure that it works // This scan should be done by user with global_admin previliges.. Ensure that it works
HTable labelsTable = null; Table labelsTable = null;
try { try {
labelsTable = new HTable(conf, LABELS_TABLE_NAME); labelsTable = new HTable(conf, LABELS_TABLE_NAME);
} catch (TableNotFoundException e) { } 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.conf.Configuration;
import org.apache.hadoop.hbase.KeyValue; import org.apache.hadoop.hbase.KeyValue;
import org.apache.hadoop.hbase.client.HTable; import org.apache.hadoop.hbase.client.HTable;
import org.apache.hadoop.hbase.client.Table;
import org.apache.hadoop.hbase.io.ImmutableBytesWritable; import org.apache.hadoop.hbase.io.ImmutableBytesWritable;
import org.apache.hadoop.hbase.io.compress.Compression.Algorithm; import org.apache.hadoop.hbase.io.compress.Compression.Algorithm;
import org.apache.hadoop.hbase.io.encoding.DataBlockEncoding; import org.apache.hadoop.hbase.io.encoding.DataBlockEncoding;
@ -160,7 +161,7 @@ public class HFileOutputFormat extends FileOutputFormat<ImmutableBytesWritable,
@edu.umd.cs.findbugs.annotations.SuppressWarnings( @edu.umd.cs.findbugs.annotations.SuppressWarnings(
value="RCN_REDUNDANT_NULLCHECK_OF_NONNULL_VALUE") value="RCN_REDUNDANT_NULLCHECK_OF_NONNULL_VALUE")
@VisibleForTesting @VisibleForTesting
static void configureCompression(HTable table, Configuration conf) throws IOException { static void configureCompression(Table table, Configuration conf) throws IOException {
HFileOutputFormat2.configureCompression(table, conf); HFileOutputFormat2.configureCompression(table, conf);
} }
@ -174,7 +175,7 @@ public class HFileOutputFormat extends FileOutputFormat<ImmutableBytesWritable,
* on failure to read column family descriptors * on failure to read column family descriptors
*/ */
@VisibleForTesting @VisibleForTesting
static void configureBlockSize(HTable table, Configuration conf) throws IOException { static void configureBlockSize(Table table, Configuration conf) throws IOException {
HFileOutputFormat2.configureBlockSize(table, conf); HFileOutputFormat2.configureBlockSize(table, conf);
} }
@ -188,7 +189,7 @@ public class HFileOutputFormat extends FileOutputFormat<ImmutableBytesWritable,
* on failure to read column family descriptors * on failure to read column family descriptors
*/ */
@VisibleForTesting @VisibleForTesting
static void configureBloomType(HTable table, Configuration conf) throws IOException { static void configureBloomType(Table table, Configuration conf) throws IOException {
HFileOutputFormat2.configureBloomType(table, conf); HFileOutputFormat2.configureBloomType(table, conf);
} }
@ -202,7 +203,7 @@ public class HFileOutputFormat extends FileOutputFormat<ImmutableBytesWritable,
* on failure to read column family descriptors * on failure to read column family descriptors
*/ */
@VisibleForTesting @VisibleForTesting
static void configureDataBlockEncoding(HTable table, static void configureDataBlockEncoding(Table table,
Configuration conf) throws IOException { Configuration conf) throws IOException {
HFileOutputFormat2.configureDataBlockEncoding(table, conf); 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.KeyValueUtil;
import org.apache.hadoop.hbase.client.HTable; import org.apache.hadoop.hbase.client.HTable;
import org.apache.hadoop.hbase.client.Put; import org.apache.hadoop.hbase.client.Put;
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.ImmutableBytesWritable;
import org.apache.hadoop.hbase.io.compress.Compression; import org.apache.hadoop.hbase.io.compress.Compression;
import org.apache.hadoop.hbase.io.compress.Compression.Algorithm; 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, * Return the start keys of all of the regions in this table,
* as a list of ImmutableBytesWritable. * as a list of ImmutableBytesWritable.
*/ */
private static List<ImmutableBytesWritable> getRegionStartKeys(HTable table) private static List<ImmutableBytesWritable> getRegionStartKeys(RegionLocator table)
throws IOException { throws IOException {
byte[][] byteKeys = table.getStartKeys(); byte[][] byteKeys = table.getStartKeys();
ArrayList<ImmutableBytesWritable> ret = ArrayList<ImmutableBytesWritable> ret =
@ -544,7 +546,7 @@ public class HFileOutputFormat2
value="RCN_REDUNDANT_NULLCHECK_OF_NONNULL_VALUE") value="RCN_REDUNDANT_NULLCHECK_OF_NONNULL_VALUE")
@VisibleForTesting @VisibleForTesting
static void configureCompression( static void configureCompression(
HTable table, Configuration conf) throws IOException { Table table, Configuration conf) throws IOException {
StringBuilder compressionConfigValue = new StringBuilder(); StringBuilder compressionConfigValue = new StringBuilder();
HTableDescriptor tableDescriptor = table.getTableDescriptor(); HTableDescriptor tableDescriptor = table.getTableDescriptor();
if(tableDescriptor == null){ if(tableDescriptor == null){
@ -578,7 +580,7 @@ public class HFileOutputFormat2
*/ */
@VisibleForTesting @VisibleForTesting
static void configureBlockSize( static void configureBlockSize(
HTable table, Configuration conf) throws IOException { Table table, Configuration conf) throws IOException {
StringBuilder blockSizeConfigValue = new StringBuilder(); StringBuilder blockSizeConfigValue = new StringBuilder();
HTableDescriptor tableDescriptor = table.getTableDescriptor(); HTableDescriptor tableDescriptor = table.getTableDescriptor();
if (tableDescriptor == null) { if (tableDescriptor == null) {
@ -612,7 +614,7 @@ public class HFileOutputFormat2
*/ */
@VisibleForTesting @VisibleForTesting
static void configureBloomType( static void configureBloomType(
HTable table, Configuration conf) throws IOException { Table table, Configuration conf) throws IOException {
HTableDescriptor tableDescriptor = table.getTableDescriptor(); HTableDescriptor tableDescriptor = table.getTableDescriptor();
if (tableDescriptor == null) { if (tableDescriptor == null) {
// could happen with mock table instance // could happen with mock table instance
@ -647,7 +649,7 @@ public class HFileOutputFormat2
* on failure to read column family descriptors * on failure to read column family descriptors
*/ */
@VisibleForTesting @VisibleForTesting
static void configureDataBlockEncoding(HTable table, static void configureDataBlockEncoding(Table table,
Configuration conf) throws IOException { Configuration conf) throws IOException {
HTableDescriptor tableDescriptor = table.getTableDescriptor(); HTableDescriptor tableDescriptor = table.getTableDescriptor();
if (tableDescriptor == null) { 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.conf.Configuration;
import org.apache.hadoop.hbase.HBaseConfiguration; import org.apache.hadoop.hbase.HBaseConfiguration;
import org.apache.hadoop.hbase.client.HTable; import org.apache.hadoop.hbase.client.HTable;
import org.apache.hadoop.hbase.client.RegionLocator;
import org.apache.hadoop.hbase.io.ImmutableBytesWritable; import org.apache.hadoop.hbase.io.ImmutableBytesWritable;
import org.apache.hadoop.hbase.util.Bytes; import org.apache.hadoop.hbase.util.Bytes;
import org.apache.hadoop.mapreduce.Partitioner; import org.apache.hadoop.mapreduce.Partitioner;
@ -53,7 +54,7 @@ implements Configurable {
private static final Log LOG = LogFactory.getLog(HRegionPartitioner.class); private static final Log LOG = LogFactory.getLog(HRegionPartitioner.class);
private Configuration conf = null; private Configuration conf = null;
private HTable table; private RegionLocator table;
private byte[][] startKeys; 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.HConstants;
import org.apache.hadoop.hbase.HTableDescriptor; import org.apache.hadoop.hbase.HTableDescriptor;
import org.apache.hadoop.hbase.TableName; import org.apache.hadoop.hbase.TableName;
import org.apache.hadoop.hbase.client.Admin;
import org.apache.hadoop.hbase.client.HBaseAdmin; import org.apache.hadoop.hbase.client.HBaseAdmin;
import org.apache.hadoop.hbase.client.HTable; import org.apache.hadoop.hbase.client.HTable;
import org.apache.hadoop.hbase.client.Put; import org.apache.hadoop.hbase.client.Put;
@ -469,7 +470,7 @@ public class ImportTsv extends Configured implements Tool {
return job; return job;
} }
private static void createTable(HBaseAdmin admin, String tableName, String[] columns) private static void createTable(Admin admin, String tableName, String[] columns)
throws IOException { throws IOException {
HTableDescriptor htd = new HTableDescriptor(TableName.valueOf(tableName)); HTableDescriptor htd = new HTableDescriptor(TableName.valueOf(tableName));
Set<String> cfSet = new HashSet<String>(); 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.LinkedBlockingQueue;
import java.util.concurrent.ThreadPoolExecutor; import java.util.concurrent.ThreadPoolExecutor;
import java.util.concurrent.TimeUnit; import java.util.concurrent.TimeUnit;
import java.util.concurrent.atomic.AtomicLong;
import org.apache.commons.lang.mutable.MutableInt; import org.apache.commons.lang.mutable.MutableInt;
import org.apache.commons.logging.Log; 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.KeyValueUtil;
import org.apache.hadoop.hbase.TableName; import org.apache.hadoop.hbase.TableName;
import org.apache.hadoop.hbase.TableNotFoundException; import org.apache.hadoop.hbase.TableNotFoundException;
import org.apache.hadoop.hbase.client.Admin;
import org.apache.hadoop.hbase.client.HBaseAdmin; import org.apache.hadoop.hbase.client.HBaseAdmin;
import org.apache.hadoop.hbase.client.HConnection; import org.apache.hadoop.hbase.client.HConnection;
import org.apache.hadoop.hbase.client.HTable; import org.apache.hadoop.hbase.client.HTable;
import org.apache.hadoop.hbase.client.RegionServerCallable; import org.apache.hadoop.hbase.client.RegionServerCallable;
import org.apache.hadoop.hbase.client.RpcRetryingCallerFactory; 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.client.coprocessor.SecureBulkLoadClient;
import org.apache.hadoop.hbase.io.HFileLink; import org.apache.hadoop.hbase.io.HFileLink;
import org.apache.hadoop.hbase.io.HalfStoreFileReader; import org.apache.hadoop.hbase.io.HalfStoreFileReader;
@ -106,7 +107,7 @@ import java.util.UUID;
@InterfaceStability.Stable @InterfaceStability.Stable
public class LoadIncrementalHFiles extends Configured implements Tool { public class LoadIncrementalHFiles extends Configured implements Tool {
private static final Log LOG = LogFactory.getLog(LoadIncrementalHFiles.class); 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 NAME = "completebulkload";
public static final String MAX_FILES_PER_REGION_PER_FAMILY 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 * them. Any failures are re-queued for another pass with the
* groupOrSplitPhase. * groupOrSplitPhase.
*/ */
protected void bulkLoadPhase(final HTable table, final HConnection conn, protected void bulkLoadPhase(final Table table, final HConnection conn,
ExecutorService pool, Deque<LoadQueueItem> queue, ExecutorService pool, Deque<LoadQueueItem> queue,
final Multimap<ByteBuffer, LoadQueueItem> regionGroups) throws IOException { final Multimap<ByteBuffer, LoadQueueItem> regionGroups) throws IOException {
// atomically bulk load the groups. // atomically bulk load the groups.
@ -482,7 +483,7 @@ public class LoadIncrementalHFiles extends Configured implements Tool {
} }
protected List<LoadQueueItem> splitStoreFile(final LoadQueueItem item, protected List<LoadQueueItem> splitStoreFile(final LoadQueueItem item,
final HTable table, byte[] startKey, final Table table, byte[] startKey,
byte[] splitKey) throws IOException { byte[] splitKey) throws IOException {
final Path hfilePath = item.hfilePath; final Path hfilePath = item.hfilePath;
@ -646,7 +647,7 @@ public class LoadIncrementalHFiles extends Configured implements Tool {
if(!userProvider.isHBaseSecurityEnabled()) { if(!userProvider.isHBaseSecurityEnabled()) {
success = ProtobufUtil.bulkLoadHFile(getStub(), famPaths, regionName, assignSeqIds); success = ProtobufUtil.bulkLoadHFile(getStub(), famPaths, regionName, assignSeqIds);
} else { } else {
HTable table = new HTable(conn.getConfiguration(), getTableName()); Table table = new HTable(conn.getConfiguration(), getTableName());
secureClient = new SecureBulkLoadClient(table); secureClient = new SecureBulkLoadClient(table);
success = secureClient.bulkLoadHFiles(famPaths, fsDelegationToken.getUserToken(), success = secureClient.bulkLoadHFiles(famPaths, fsDelegationToken.getUserToken(),
bulkToken, getLocation().getRegionInfo().getStartKey()); 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.HTable;
import org.apache.hadoop.hbase.client.Result; import org.apache.hadoop.hbase.client.Result;
import org.apache.hadoop.hbase.client.Scan; import org.apache.hadoop.hbase.client.Scan;
import org.apache.hadoop.hbase.client.Table;
import org.apache.hadoop.hbase.io.ImmutableBytesWritable; import org.apache.hadoop.hbase.io.ImmutableBytesWritable;
import org.apache.hadoop.hbase.util.Bytes; import org.apache.hadoop.hbase.util.Bytes;
import org.apache.hadoop.hbase.util.Pair; 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" + " previous error. Please look at the previous logs lines from"
+ " the task's full log for more details."); + " the task's full log for more details.");
} }
HTable table = Table table =
new HTable(context.getConfiguration(), tSplit.getTableName()); new HTable(context.getConfiguration(), tSplit.getTableName());
TableRecordReader trr = this.tableRecordReader; TableRecordReader trr = this.tableRecordReader;

View File

@ -19,10 +19,8 @@
package org.apache.hadoop.hbase.mapreduce; package org.apache.hadoop.hbase.mapreduce;
import java.io.IOException; import java.io.IOException;
import java.io.InterruptedIOException;
import java.net.InetAddress; import java.net.InetAddress;
import java.net.InetSocketAddress; import java.net.InetSocketAddress;
import java.text.MessageFormat;
import java.util.ArrayList; import java.util.ArrayList;
import java.util.HashMap; import java.util.HashMap;
import java.util.List; 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.HTable;
import org.apache.hadoop.hbase.client.Mutation; import org.apache.hadoop.hbase.client.Mutation;
import org.apache.hadoop.hbase.client.Put; import org.apache.hadoop.hbase.client.Put;
import org.apache.hadoop.hbase.client.Table;
import org.apache.hadoop.hbase.zookeeper.ZKUtil; import org.apache.hadoop.hbase.zookeeper.ZKUtil;
import org.apache.hadoop.mapreduce.JobContext; import org.apache.hadoop.mapreduce.JobContext;
import org.apache.hadoop.mapreduce.OutputCommitter; import org.apache.hadoop.mapreduce.OutputCommitter;
@ -88,14 +89,14 @@ implements Configurable {
extends RecordWriter<KEY, Mutation> { extends RecordWriter<KEY, Mutation> {
/** The table to write to. */ /** The table to write to. */
private HTable table; private Table table;
/** /**
* Instantiate a TableRecordWriter with the HBase HClient for writing. * Instantiate a TableRecordWriter with the HBase HClient for writing.
* *
* @param table The table to write to. * @param table The table to write to.
*/ */
public TableRecordWriter(HTable table) { public TableRecordWriter(Table table) {
this.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.HTable;
import org.apache.hadoop.hbase.client.Result; import org.apache.hadoop.hbase.client.Result;
import org.apache.hadoop.hbase.client.Scan; import org.apache.hadoop.hbase.client.Scan;
import org.apache.hadoop.hbase.client.Table;
import org.apache.hadoop.hbase.io.ImmutableBytesWritable; import org.apache.hadoop.hbase.io.ImmutableBytesWritable;
import org.apache.hadoop.mapreduce.InputSplit; import org.apache.hadoop.mapreduce.InputSplit;
import org.apache.hadoop.mapreduce.RecordReader; import org.apache.hadoop.mapreduce.RecordReader;
@ -57,7 +58,7 @@ extends RecordReader<ImmutableBytesWritable, Result> {
* *
* @param htable The {@link HTable} to scan. * @param htable The {@link HTable} to scan.
*/ */
public void setHTable(HTable htable) { public void setHTable(Table htable) {
this.recordReaderImpl.setHTable(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.ResultScanner;
import org.apache.hadoop.hbase.client.Scan; import org.apache.hadoop.hbase.client.Scan;
import org.apache.hadoop.hbase.client.ScannerCallable; import org.apache.hadoop.hbase.client.ScannerCallable;
import org.apache.hadoop.hbase.client.Table;
import org.apache.hadoop.hbase.client.metrics.ScanMetrics; import org.apache.hadoop.hbase.client.metrics.ScanMetrics;
import org.apache.hadoop.hbase.io.ImmutableBytesWritable; import org.apache.hadoop.hbase.io.ImmutableBytesWritable;
import org.apache.hadoop.hbase.protobuf.ProtobufUtil; import org.apache.hadoop.hbase.protobuf.ProtobufUtil;
@ -58,7 +59,7 @@ public class TableRecordReaderImpl {
private ResultScanner scanner = null; private ResultScanner scanner = null;
private Scan scan = null; private Scan scan = null;
private Scan currentScan = null; private Scan currentScan = null;
private HTable htable = null; private Table htable = null;
private byte[] lastSuccessfulRow = null; private byte[] lastSuccessfulRow = null;
private ImmutableBytesWritable key = null; private ImmutableBytesWritable key = null;
private Result value = null; private Result value = null;
@ -121,7 +122,7 @@ public class TableRecordReaderImpl {
* *
* @param htable The {@link HTable} to scan. * @param htable The {@link HTable} to scan.
*/ */
public void setHTable(HTable htable) { public void setHTable(Table htable) {
Configuration conf = htable.getConfiguration(); Configuration conf = htable.getConfiguration();
logScannerActivity = conf.getBoolean( logScannerActivity = conf.getBoolean(
ScannerCallable.LOG_SCANNER_ACTIVITY, false); 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.Result;
import org.apache.hadoop.hbase.client.ResultScanner; import org.apache.hadoop.hbase.client.ResultScanner;
import org.apache.hadoop.hbase.client.Scan; import org.apache.hadoop.hbase.client.Scan;
import org.apache.hadoop.hbase.client.Table;
import org.apache.hadoop.hbase.io.ImmutableBytesWritable; import org.apache.hadoop.hbase.io.ImmutableBytesWritable;
import org.apache.hadoop.hbase.mapreduce.TableInputFormat; import org.apache.hadoop.hbase.mapreduce.TableInputFormat;
import org.apache.hadoop.hbase.mapreduce.TableMapReduceUtil; import org.apache.hadoop.hbase.mapreduce.TableMapReduceUtil;
@ -120,7 +121,7 @@ public class VerifyReplication extends Configured implements Tool {
Configuration peerConf = HBaseConfiguration.create(conf); Configuration peerConf = HBaseConfiguration.create(conf);
ZKUtil.applyClusterKeyToConf(peerConf, zkClusterKey); 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()); scan.setStartRow(value.getRow());
replicatedScanner = replicatedTable.getScanner(scan); replicatedScanner = replicatedTable.getScanner(scan);
return null; return null;

View File

@ -30,7 +30,6 @@ import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory; import org.apache.commons.logging.LogFactory;
import org.apache.hadoop.classification.InterfaceAudience; import org.apache.hadoop.classification.InterfaceAudience;
import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.fs.FileSystem;
import org.apache.hadoop.hbase.CoordinatedStateManager; import org.apache.hadoop.hbase.CoordinatedStateManager;
import org.apache.hadoop.hbase.CoordinatedStateManagerFactory; import org.apache.hadoop.hbase.CoordinatedStateManagerFactory;
import org.apache.hadoop.hbase.MasterNotRunningException; 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.HConstants;
import org.apache.hadoop.hbase.LocalHBaseCluster; import org.apache.hadoop.hbase.LocalHBaseCluster;
import org.apache.hadoop.hbase.ZooKeeperConnectionException; import org.apache.hadoop.hbase.ZooKeeperConnectionException;
import org.apache.hadoop.hbase.client.Admin;
import org.apache.hadoop.hbase.client.HBaseAdmin; import org.apache.hadoop.hbase.client.HBaseAdmin;
import org.apache.hadoop.hbase.regionserver.HRegionServer; import org.apache.hadoop.hbase.regionserver.HRegionServer;
import org.apache.hadoop.hbase.util.JVMClusterUtil; import org.apache.hadoop.hbase.util.JVMClusterUtil;
@ -209,7 +209,7 @@ public class HMasterCommandLine extends ServerCommandLine {
} }
private int stopMaster() { private int stopMaster() {
HBaseAdmin adm = null; Admin adm = null;
try { try {
Configuration conf = getConf(); Configuration conf = getConf();
// Don't try more than once // 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.HRegionInfo;
import org.apache.hadoop.hbase.ServerName; import org.apache.hadoop.hbase.ServerName;
import org.apache.hadoop.hbase.TableName; import org.apache.hadoop.hbase.TableName;
import org.apache.hadoop.hbase.client.Admin;
import org.apache.hadoop.hbase.client.HBaseAdmin; import org.apache.hadoop.hbase.client.HBaseAdmin;
import org.apache.hadoop.hbase.client.HConnection; import org.apache.hadoop.hbase.client.HConnection;
import org.apache.hadoop.hbase.client.HConnectionManager; import org.apache.hadoop.hbase.client.HConnectionManager;
@ -92,7 +93,7 @@ public class RegionPlacementMaintainer {
private Configuration conf; private Configuration conf;
private final boolean enforceLocality; private final boolean enforceLocality;
private final boolean enforceMinAssignmentMove; private final boolean enforceMinAssignmentMove;
private HBaseAdmin admin; private Admin admin;
private RackManager rackManager; private RackManager rackManager;
private Set<TableName> targetTableSet; private Set<TableName> targetTableSet;
@ -127,7 +128,7 @@ public class RegionPlacementMaintainer {
* @return the cached HBaseAdmin * @return the cached HBaseAdmin
* @throws IOException * @throws IOException
*/ */
private HBaseAdmin getHBaseAdmin() throws IOException { private Admin getHBaseAdmin() throws IOException {
if (this.admin == null) { if (this.admin == null) {
this.admin = new HBaseAdmin(this.conf); 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.Put;
import org.apache.hadoop.hbase.client.Result; import org.apache.hadoop.hbase.client.Result;
import org.apache.hadoop.hbase.client.ResultScanner; import org.apache.hadoop.hbase.client.ResultScanner;
import org.apache.hadoop.hbase.client.Table;
import org.apache.hadoop.hbase.constraint.ConstraintException; import org.apache.hadoop.hbase.constraint.ConstraintException;
import org.apache.hadoop.hbase.master.handler.CreateTableHandler; import org.apache.hadoop.hbase.master.handler.CreateTableHandler;
import org.apache.hadoop.hbase.protobuf.ProtobufUtil; import org.apache.hadoop.hbase.protobuf.ProtobufUtil;
@ -107,7 +108,7 @@ public class TableNamespaceManager {
isTableAvailableAndInitialized(); isTableAvailableAndInitialized();
} }
private synchronized HTable getNamespaceTable() throws IOException { private synchronized Table getNamespaceTable() throws IOException {
if (!isTableAvailableAndInitialized()) { if (!isTableAvailableAndInitialized()) {
throw new IOException(this.getClass().getName() + " isn't ready to serve"); 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 { public synchronized void update(NamespaceDescriptor ns) throws IOException {
HTable table = getNamespaceTable(); Table table = getNamespaceTable();
if (get(table, ns.getName()) == null) { if (get(table, ns.getName()) == null) {
throw new NamespaceNotFoundException(ns.getName()); throw new NamespaceNotFoundException(ns.getName());
} }
upsert(table, ns); 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))); Result res = table.get(new Get(Bytes.toBytes(name)));
if (res.isEmpty()) { if (res.isEmpty()) {
return null; return null;
@ -144,7 +145,7 @@ public class TableNamespaceManager {
HBaseProtos.NamespaceDescriptor.parseFrom(val)); 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) { if (get(table, ns.getName()) != null) {
throw new NamespaceExistException(ns.getName()); throw new NamespaceExistException(ns.getName());
} }
@ -154,7 +155,7 @@ public class TableNamespaceManager {
upsert(table, ns); 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())); Put p = new Put(Bytes.toBytes(ns.getName()));
p.addImmutable(HTableDescriptor.NAMESPACE_FAMILY_INFO_BYTES, p.addImmutable(HTableDescriptor.NAMESPACE_FAMILY_INFO_BYTES,
HTableDescriptor.NAMESPACE_COL_DESC_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.HConnection;
import org.apache.hadoop.hbase.client.HTable; import org.apache.hadoop.hbase.client.HTable;
import org.apache.hadoop.hbase.client.Put; import org.apache.hadoop.hbase.client.Put;
import org.apache.hadoop.hbase.client.Table;
import org.apache.hadoop.hbase.master.RackManager; import org.apache.hadoop.hbase.master.RackManager;
import org.apache.hadoop.hbase.protobuf.ProtobufUtil; import org.apache.hadoop.hbase.protobuf.ProtobufUtil;
import org.apache.hadoop.hbase.protobuf.generated.HBaseProtos; import org.apache.hadoop.hbase.protobuf.generated.HBaseProtos;
@ -120,7 +121,7 @@ public class FavoredNodeAssignmentHelper {
} }
} }
// Write the region assignments to the meta table. // Write the region assignments to the meta table.
HTable metaTable = null; Table metaTable = null;
try { try {
metaTable = new HTable(conf, TableName.META_TABLE_NAME); metaTable = new HTable(conf, TableName.META_TABLE_NAME);
metaTable.put(puts); 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.Result;
import org.apache.hadoop.hbase.client.ResultScanner; import org.apache.hadoop.hbase.client.ResultScanner;
import org.apache.hadoop.hbase.client.Scan; import org.apache.hadoop.hbase.client.Scan;
import org.apache.hadoop.hbase.client.Table;
import org.apache.hadoop.hbase.executor.EventType; import org.apache.hadoop.hbase.executor.EventType;
import org.apache.hadoop.hbase.master.HMaster; import org.apache.hadoop.hbase.master.HMaster;
import org.apache.hadoop.hbase.master.MasterCoprocessorHost; import org.apache.hadoop.hbase.master.MasterCoprocessorHost;
@ -94,7 +95,7 @@ public class ModifyTableHandler extends TableEventHandler {
Set<byte[]> tableRows = new HashSet<byte[]>(); Set<byte[]> tableRows = new HashSet<byte[]>();
Scan scan = MetaTableAccessor.getScanForTableName(table); Scan scan = MetaTableAccessor.getScanForTableName(table);
scan.addColumn(HConstants.CATALOG_FAMILY, HConstants.REGIONINFO_QUALIFIER); scan.addColumn(HConstants.CATALOG_FAMILY, HConstants.REGIONINFO_QUALIFIER);
HTable htable = null; Table htable = null;
try { try {
htable = new HTable(masterServices.getConfiguration(), TableName.META_TABLE_NAME); htable = new HTable(masterServices.getConfiguration(), TableName.META_TABLE_NAME);
ResultScanner resScanner = htable.getScanner(scan); 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.Delete;
import org.apache.hadoop.hbase.client.HConnection; import org.apache.hadoop.hbase.client.HConnection;
import org.apache.hadoop.hbase.client.HConnectionManager; import org.apache.hadoop.hbase.client.HConnectionManager;
import org.apache.hadoop.hbase.client.HTableInterface;
import org.apache.hadoop.hbase.client.Mutation; import org.apache.hadoop.hbase.client.Mutation;
import org.apache.hadoop.hbase.client.Put; import org.apache.hadoop.hbase.client.Put;
import org.apache.hadoop.hbase.client.Row; 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.AdminProtos.WALEntry;
import org.apache.hadoop.hbase.protobuf.generated.HBaseProtos; import org.apache.hadoop.hbase.protobuf.generated.HBaseProtos;
@ -230,7 +230,7 @@ public class ReplicationSink {
if (allRows.isEmpty()) { if (allRows.isEmpty()) {
return; return;
} }
HTableInterface table = null; Table table = null;
try { try {
table = this.sharedHtableCon.getTable(tableName); table = this.sharedHtableCon.getTable(tableName);
for (List<Row> rows : allRows) { 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.classification.InterfaceAudience;
import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.hbase.client.HBaseAdmin; 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.filter.ParseFilter;
import org.apache.hadoop.hbase.security.UserProvider; import org.apache.hadoop.hbase.security.UserProvider;
import org.apache.hadoop.hbase.util.ConnectionCache; import org.apache.hadoop.hbase.util.ConnectionCache;
@ -100,7 +100,7 @@ public class RESTServlet implements Constants {
/** /**
* Caller closes the table afterwards. * Caller closes the table afterwards.
*/ */
HTableInterface getTable(String tableName) throws IOException { Table getTable(String tableName) throws IOException {
return connectionCache.getTable(tableName); 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.HConstants;
import org.apache.hadoop.hbase.KeyValue; import org.apache.hadoop.hbase.KeyValue;
import org.apache.hadoop.hbase.client.Delete; 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.Put;
import org.apache.hadoop.hbase.client.Table;
import org.apache.hadoop.hbase.rest.model.CellModel; import org.apache.hadoop.hbase.rest.model.CellModel;
import org.apache.hadoop.hbase.rest.model.CellSetModel; import org.apache.hadoop.hbase.rest.model.CellSetModel;
import org.apache.hadoop.hbase.rest.model.RowModel; import org.apache.hadoop.hbase.rest.model.RowModel;
@ -182,7 +182,7 @@ public class RowResource extends ResourceBase {
.build(); .build();
} }
HTableInterface table = null; Table table = null;
try { try {
List<RowModel> rows = model.getRows(); List<RowModel> rows = model.getRows();
List<Put> puts = new ArrayList<Put>(); List<Put> puts = new ArrayList<Put>();
@ -253,7 +253,7 @@ public class RowResource extends ResourceBase {
.type(MIMETYPE_TEXT).entity("Forbidden" + CRLF) .type(MIMETYPE_TEXT).entity("Forbidden" + CRLF)
.build(); .build();
} }
HTableInterface table = null; Table table = null;
try { try {
byte[] row = rowspec.getRow(); byte[] row = rowspec.getRow();
byte[][] columns = rowspec.getColumns(); byte[][] columns = rowspec.getColumns();
@ -393,7 +393,7 @@ public class RowResource extends ResourceBase {
} }
} }
} }
HTableInterface table = null; Table table = null;
try { try {
table = servlet.getTable(tableResource.getName()); table = servlet.getTable(tableResource.getName());
table.delete(delete); table.delete(delete);
@ -422,7 +422,7 @@ public class RowResource extends ResourceBase {
* @return Response 200 OK, 304 Not modified, 400 Bad request * @return Response 200 OK, 304 Not modified, 400 Bad request
*/ */
Response checkAndPut(final CellSetModel model) { Response checkAndPut(final CellSetModel model) {
HTableInterface table = null; Table table = null;
try { try {
table = servlet.getTable(tableResource.getName()); table = servlet.getTable(tableResource.getName());
if (model.getRows().size() != 1) { if (model.getRows().size() != 1) {
@ -513,7 +513,7 @@ public class RowResource extends ResourceBase {
* @return Response 200 OK, 304 Not modified, 400 Bad request * @return Response 200 OK, 304 Not modified, 400 Bad request
*/ */
Response checkAndDelete(final CellSetModel model) { Response checkAndDelete(final CellSetModel model) {
HTableInterface table = null; Table table = null;
Delete delete = null; Delete delete = null;
try { try {
table = servlet.getTable(tableResource.getName()); 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.DoNotRetryIOException;
import org.apache.hadoop.hbase.KeyValue; import org.apache.hadoop.hbase.KeyValue;
import org.apache.hadoop.hbase.client.Get; 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.Result;
import org.apache.hadoop.hbase.client.Table;
import org.apache.hadoop.hbase.filter.Filter; import org.apache.hadoop.hbase.filter.Filter;
import org.apache.hadoop.util.StringUtils; import org.apache.hadoop.util.StringUtils;
@ -45,7 +45,7 @@ public class RowResultGenerator extends ResultGenerator {
public RowResultGenerator(final String tableName, final RowSpec rowspec, public RowResultGenerator(final String tableName, final RowSpec rowspec,
final Filter filter, final boolean cacheBlocks) final Filter filter, final boolean cacheBlocks)
throws IllegalArgumentException, IOException { throws IllegalArgumentException, IOException {
HTableInterface table = RESTServlet.getInstance().getTable(tableName); Table table = RESTServlet.getInstance().getTable(tableName);
try { try {
Get get = new Get(rowspec.getRow()); Get get = new Get(rowspec.getRow());
if (rowspec.hasColumns()) { 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.Cell;
import org.apache.hadoop.hbase.KeyValue; import org.apache.hadoop.hbase.KeyValue;
import org.apache.hadoop.hbase.UnknownScannerException; 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.Result;
import org.apache.hadoop.hbase.client.ResultScanner; import org.apache.hadoop.hbase.client.ResultScanner;
import org.apache.hadoop.hbase.client.Scan; import org.apache.hadoop.hbase.client.Scan;
import org.apache.hadoop.hbase.client.Table;
import org.apache.hadoop.hbase.filter.Filter; import org.apache.hadoop.hbase.filter.Filter;
import org.apache.hadoop.hbase.rest.model.ScannerModel; import org.apache.hadoop.hbase.rest.model.ScannerModel;
import org.apache.hadoop.hbase.security.visibility.Authorizations; 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, public ScannerResultGenerator(final String tableName, final RowSpec rowspec,
final Filter filter, final int caching, final boolean cacheBlocks) final Filter filter, final int caching, final boolean cacheBlocks)
throws IllegalArgumentException, IOException { throws IllegalArgumentException, IOException {
HTableInterface table = RESTServlet.getInstance().getTable(tableName); Table table = RESTServlet.getInstance().getTable(tableName);
try { try {
Scan scan; Scan scan;
if (rowspec.hasEndRow()) { 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.TableName;
import org.apache.hadoop.hbase.TableNotFoundException; import org.apache.hadoop.hbase.TableNotFoundException;
import org.apache.hadoop.hbase.client.HBaseAdmin; 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.ColumnSchemaModel;
import org.apache.hadoop.hbase.rest.model.TableSchemaModel; import org.apache.hadoop.hbase.rest.model.TableSchemaModel;
import org.apache.hadoop.hbase.util.Bytes; import org.apache.hadoop.hbase.util.Bytes;
@ -74,7 +74,7 @@ public class SchemaResource extends ResourceBase {
private HTableDescriptor getTableSchema() throws IOException, private HTableDescriptor getTableSchema() throws IOException,
TableNotFoundException { TableNotFoundException {
HTableInterface table = servlet.getTable(tableResource.getName()); Table table = servlet.getTable(tableResource.getName());
try { try {
return table.getTableDescriptor(); return table.getTableDescriptor();
} finally { } finally {

View File

@ -35,8 +35,8 @@ import org.apache.commons.lang.StringUtils;
import org.apache.commons.logging.Log; import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory; import org.apache.commons.logging.LogFactory;
import org.apache.hadoop.classification.InterfaceAudience; 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.Scan;
import org.apache.hadoop.hbase.client.Table;
import org.apache.hadoop.hbase.filter.Filter; import org.apache.hadoop.hbase.filter.Filter;
import org.apache.hadoop.hbase.filter.FilterList; import org.apache.hadoop.hbase.filter.FilterList;
import org.apache.hadoop.hbase.filter.ParseFilter; 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 Row => " + endRow + " Columns => " + column + " Start Time => " + startTime
+ " End Time => " + endTime + " Cache Blocks => " + cacheBlocks + " Max Versions => " + " End Time => " + endTime + " Cache Blocks => " + cacheBlocks + " Max Versions => "
+ maxVersions + " Batch Size => " + batchSize); + maxVersions + " Batch Size => " + batchSize);
HTableInterface hTable = RESTServlet.getInstance().getTable(this.table); Table hTable = RESTServlet.getInstance().getTable(this.table);
Scan tableScan = new Scan(); Scan tableScan = new Scan();
tableScan.setBatch(batchSize); tableScan.setBatch(batchSize);
tableScan.setMaxVersions(maxVersions); 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.Result;
import org.apache.hadoop.hbase.client.ResultScanner; import org.apache.hadoop.hbase.client.ResultScanner;
import org.apache.hadoop.hbase.client.Scan; import org.apache.hadoop.hbase.client.Scan;
import org.apache.hadoop.hbase.client.Table;
import org.apache.hadoop.hbase.exceptions.DeserializationException; import org.apache.hadoop.hbase.exceptions.DeserializationException;
import org.apache.hadoop.hbase.filter.CompareFilter.CompareOp; import org.apache.hadoop.hbase.filter.CompareFilter.CompareOp;
import org.apache.hadoop.hbase.filter.QualifierFilter; import org.apache.hadoop.hbase.filter.QualifierFilter;
@ -172,7 +173,7 @@ public class AccessControlLists {
Bytes.toString(key)+": "+Bytes.toStringBinary(value) Bytes.toString(key)+": "+Bytes.toStringBinary(value)
); );
} }
HTable acls = null; Table acls = null;
try { try {
acls = new HTable(conf, ACL_TABLE_NAME); acls = new HTable(conf, ACL_TABLE_NAME);
acls.put(p); acls.put(p);
@ -203,7 +204,7 @@ public class AccessControlLists {
LOG.debug("Removing permission "+ userPerm.toString()); LOG.debug("Removing permission "+ userPerm.toString());
} }
d.deleteColumns(ACL_LIST_FAMILY, key); d.deleteColumns(ACL_LIST_FAMILY, key);
HTable acls = null; Table acls = null;
try { try {
acls = new HTable(conf, ACL_TABLE_NAME); acls = new HTable(conf, ACL_TABLE_NAME);
acls.delete(d); acls.delete(d);
@ -223,7 +224,7 @@ public class AccessControlLists {
LOG.debug("Removing permissions of removed table "+ tableName); LOG.debug("Removing permissions of removed table "+ tableName);
} }
HTable acls = null; Table acls = null;
try { try {
acls = new HTable(conf, ACL_TABLE_NAME); acls = new HTable(conf, ACL_TABLE_NAME);
acls.delete(d); acls.delete(d);
@ -243,7 +244,7 @@ public class AccessControlLists {
LOG.debug("Removing permissions of removed namespace "+ namespace); LOG.debug("Removing permissions of removed namespace "+ namespace);
} }
HTable acls = null; Table acls = null;
try { try {
acls = new HTable(conf, ACL_TABLE_NAME); acls = new HTable(conf, ACL_TABLE_NAME);
acls.delete(d); acls.delete(d);
@ -263,7 +264,7 @@ public class AccessControlLists {
" from table "+ tableName); " from table "+ tableName);
} }
HTable acls = null; Table acls = null;
try { try {
acls = new HTable(conf, ACL_TABLE_NAME); acls = new HTable(conf, ACL_TABLE_NAME);
@ -424,7 +425,7 @@ public class AccessControlLists {
Scan scan = new Scan(); Scan scan = new Scan();
scan.addFamily(ACL_LIST_FAMILY); scan.addFamily(ACL_LIST_FAMILY);
HTable acls = null; Table acls = null;
ResultScanner scanner = null; ResultScanner scanner = null;
try { try {
acls = new HTable(conf, ACL_TABLE_NAME); 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_ // for normal user tables, we just read the table row from _acl_
ListMultimap<String, TablePermission> perms = ArrayListMultimap.create(); ListMultimap<String, TablePermission> perms = ArrayListMultimap.create();
HTable acls = null; Table acls = null;
try { try {
acls = new HTable(conf, ACL_TABLE_NAME); acls = new HTable(conf, ACL_TABLE_NAME);
Get get = new Get(entryName); 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.HConstants;
import org.apache.hadoop.hbase.TableName; import org.apache.hadoop.hbase.TableName;
import org.apache.hadoop.hbase.client.HTable; import org.apache.hadoop.hbase.client.HTable;
import org.apache.hadoop.hbase.client.Table;
import org.apache.hadoop.hbase.ipc.CoprocessorRpcChannel; import org.apache.hadoop.hbase.ipc.CoprocessorRpcChannel;
import org.apache.hadoop.hbase.protobuf.ProtobufUtil; import org.apache.hadoop.hbase.protobuf.ProtobufUtil;
import org.apache.hadoop.hbase.protobuf.generated.AuthenticationProtos; import org.apache.hadoop.hbase.protobuf.generated.AuthenticationProtos;
@ -53,7 +54,7 @@ public class TokenUtil {
*/ */
public static Token<AuthenticationTokenIdentifier> obtainToken( public static Token<AuthenticationTokenIdentifier> obtainToken(
Configuration conf) throws IOException { Configuration conf) throws IOException {
HTable meta = null; Table meta = null;
try { try {
meta = new HTable(conf, TableName.META_TABLE_NAME); meta = new HTable(conf, TableName.META_TABLE_NAME);
CoprocessorRpcChannel rpcChannel = meta.coprocessorService(HConstants.EMPTY_START_ROW); 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.HTable;
import org.apache.hadoop.hbase.client.ResultScanner; import org.apache.hadoop.hbase.client.ResultScanner;
import org.apache.hadoop.hbase.client.Scan; import org.apache.hadoop.hbase.client.Scan;
import org.apache.hadoop.hbase.client.Table;
import org.apache.hadoop.util.Tool; import org.apache.hadoop.util.Tool;
import org.apache.hadoop.util.ToolRunner; import org.apache.hadoop.util.ToolRunner;
@ -326,7 +327,7 @@ public final class Canary implements Tool {
public static abstract class Monitor implements Runnable { public static abstract class Monitor implements Runnable {
protected Configuration config; protected Configuration config;
protected HBaseAdmin admin; protected Admin admin;
protected String[] targets; protected String[] targets;
protected boolean useRegExp; protected boolean useRegExp;
protected boolean initialized = false; 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) private static void sniff(final Admin admin, final Sink sink, HTableDescriptor tableDesc)
throws Exception { throws Exception {
HTable table = null; Table table = null;
try { try {
table = new HTable(admin.getConfiguration(), tableDesc.getName()); table = new HTable(admin.getConfiguration(), tableDesc.getName());
@ -506,7 +507,7 @@ public final class Canary implements Tool {
final Admin admin, final Admin admin,
final Sink sink, final Sink sink,
HRegionInfo region, HRegionInfo region,
HTable table) throws Exception { Table table) throws Exception {
HTableDescriptor tableDesc = table.getTableDescriptor(); HTableDescriptor tableDesc = table.getTableDescriptor();
byte[] startKey = null; byte[] startKey = null;
Get get = null; Get get = null;
@ -607,7 +608,7 @@ public final class Canary implements Tool {
String serverName = null; String serverName = null;
String tableName = null; String tableName = null;
HRegionInfo region = null; HRegionInfo region = null;
HTable table = null; Table table = null;
Get get = null; Get get = null;
byte[] startKey = null; byte[] startKey = null;
Scan scan = null; Scan scan = null;

View File

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

View File

@ -34,9 +34,10 @@ import org.apache.hadoop.hbase.TableName;
import org.apache.hadoop.hbase.ZooKeeperConnectionException; import org.apache.hadoop.hbase.ZooKeeperConnectionException;
import org.apache.hadoop.hbase.MetaTableAccessor; import org.apache.hadoop.hbase.MetaTableAccessor;
import org.apache.hadoop.hbase.client.Admin; import org.apache.hadoop.hbase.client.Admin;
import org.apache.hadoop.hbase.client.HBaseAdmin;
import org.apache.hadoop.hbase.client.HConnection; import org.apache.hadoop.hbase.client.HConnection;
import org.apache.hadoop.hbase.client.HTable; import org.apache.hadoop.hbase.client.HTable;
import org.apache.hadoop.hbase.client.Put;
import org.apache.hadoop.hbase.client.Table;
import org.apache.hadoop.hbase.master.RegionState; import org.apache.hadoop.hbase.master.RegionState;
import org.apache.hadoop.hbase.protobuf.ProtobufUtil; import org.apache.hadoop.hbase.protobuf.ProtobufUtil;
import org.apache.hadoop.hbase.protobuf.generated.AdminProtos.AdminService; import org.apache.hadoop.hbase.protobuf.generated.AdminProtos.AdminService;
@ -60,7 +61,7 @@ public class HBaseFsckRepair {
* @param region Region to undeploy * @param region Region to undeploy
* @param servers list of Servers to undeploy from * @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) List<ServerName> servers)
throws IOException, KeeperException, InterruptedException { throws IOException, KeeperException, InterruptedException {
HRegionInfo actualRegion = new HRegionInfo(region); HRegionInfo actualRegion = new HRegionInfo(region);
@ -86,7 +87,7 @@ public class HBaseFsckRepair {
* @throws IOException * @throws IOException
* @throws KeeperException * @throws KeeperException
*/ */
public static void fixUnassigned(HBaseAdmin admin, HRegionInfo region) public static void fixUnassigned(Admin admin, HRegionInfo region)
throws IOException, KeeperException { throws IOException, KeeperException {
HRegionInfo actualRegion = new HRegionInfo(region); HRegionInfo actualRegion = new HRegionInfo(region);
@ -106,7 +107,7 @@ public class HBaseFsckRepair {
* side-effect of requiring a HRegionInfo that considers regionId (timestamp) * side-effect of requiring a HRegionInfo that considers regionId (timestamp)
* in comparators that is addressed by HBASE-5563. * 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 { throws ZooKeeperConnectionException, KeeperException, IOException {
admin.assign(region.getRegionName()); admin.assign(region.getRegionName());
} }
@ -114,7 +115,7 @@ public class HBaseFsckRepair {
/* /*
* Should we check all assignments or just not in RIT? * 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 { HRegionInfo region) throws IOException, InterruptedException {
long timeout = admin.getConfiguration().getLong("hbase.hbck.assign.timeout", 120000); long timeout = admin.getConfiguration().getLong("hbase.hbck.assign.timeout", 120000);
long expiration = timeout + System.currentTimeMillis(); long expiration = timeout + System.currentTimeMillis();
@ -175,7 +176,7 @@ public class HBaseFsckRepair {
*/ */
public static void fixMetaHoleOnline(Configuration conf, public static void fixMetaHoleOnline(Configuration conf,
HRegionInfo hri) throws IOException { HRegionInfo hri) throws IOException {
HTable meta = new HTable(conf, TableName.META_TABLE_NAME); Table meta = new HTable(conf, TableName.META_TABLE_NAME);
MetaTableAccessor.addRegionToMeta(meta, hri); MetaTableAccessor.addRegionToMeta(meta, hri);
meta.close(); meta.close();
} }

View File

@ -37,6 +37,7 @@ import org.apache.hadoop.hbase.HTableDescriptor;
import org.apache.hadoop.hbase.RemoteExceptionHandler; import org.apache.hadoop.hbase.RemoteExceptionHandler;
import org.apache.hadoop.hbase.TableNotDisabledException; import org.apache.hadoop.hbase.TableNotDisabledException;
import org.apache.hadoop.hbase.MetaTableAccessor; 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.Delete;
import org.apache.hadoop.hbase.client.HBaseAdmin; import org.apache.hadoop.hbase.client.HBaseAdmin;
import org.apache.hadoop.hbase.client.HConnectable; import org.apache.hadoop.hbase.client.HConnectable;
@ -45,6 +46,7 @@ import org.apache.hadoop.hbase.client.HConnectionManager;
import org.apache.hadoop.hbase.client.HTable; import org.apache.hadoop.hbase.client.HTable;
import org.apache.hadoop.hbase.client.Result; import org.apache.hadoop.hbase.client.Result;
import org.apache.hadoop.hbase.client.ResultScanner; import org.apache.hadoop.hbase.client.ResultScanner;
import org.apache.hadoop.hbase.client.Table;
import org.apache.hadoop.hbase.regionserver.HRegion; import org.apache.hadoop.hbase.regionserver.HRegion;
import org.apache.hadoop.hbase.regionserver.wal.HLog; import org.apache.hadoop.hbase.regionserver.wal.HLog;
import org.apache.hadoop.hbase.regionserver.wal.HLogFactory; import org.apache.hadoop.hbase.regionserver.wal.HLogFactory;
@ -124,7 +126,7 @@ class HMerge {
throw new IllegalStateException( throw new IllegalStateException(
"HBase instance must be running to merge a normal table"); "HBase instance must be running to merge a normal table");
} }
HBaseAdmin admin = new HBaseAdmin(conf); Admin admin = new HBaseAdmin(conf);
try { try {
if (!admin.isTableDisabled(tableName)) { if (!admin.isTableDisabled(tableName)) {
throw new TableNotDisabledException(tableName); throw new TableNotDisabledException(tableName);
@ -231,7 +233,7 @@ class HMerge {
/** Instantiated to compact a normal user table */ /** Instantiated to compact a normal user table */
private static class OnlineMerger extends Merger { private static class OnlineMerger extends Merger {
private final TableName tableName; private final TableName tableName;
private final HTable table; private final Table table;
private final ResultScanner metaScanner; private final ResultScanner metaScanner;
private HRegionInfo latestRegion; 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.RegionLoad;
import org.apache.hadoop.hbase.ServerLoad; import org.apache.hadoop.hbase.ServerLoad;
import org.apache.hadoop.hbase.ServerName; 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.HBaseAdmin;
import org.apache.hadoop.hbase.client.HTable; import org.apache.hadoop.hbase.client.HTable;
import java.io.IOException; import java.io.IOException;
import java.text.MessageFormat;
import java.util.Arrays; import java.util.Arrays;
import java.util.Collection; import java.util.Collection;
import java.util.Collections; import java.util.Collections;
@ -65,7 +65,7 @@ public class RegionSizeCalculator {
} }
/** ctor for unit testing */ /** ctor for unit testing */
RegionSizeCalculator (HTable table, HBaseAdmin admin) throws IOException { RegionSizeCalculator (HTable table, Admin admin) throws IOException {
try { try {
if (!enabled(table.getConfiguration())) { 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.Result;
import org.apache.hadoop.hbase.client.ResultScanner; import org.apache.hadoop.hbase.client.ResultScanner;
import org.apache.hadoop.hbase.client.Scan; import org.apache.hadoop.hbase.client.Scan;
import org.apache.hadoop.hbase.client.Table;
import org.apache.hadoop.hbase.regionserver.HRegion; import org.apache.hadoop.hbase.regionserver.HRegion;
import org.apache.hadoop.hbase.regionserver.InternalScanner; import org.apache.hadoop.hbase.regionserver.InternalScanner;
import org.apache.hadoop.hbase.util.Bytes; 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} * A class that makes a {@link Incommon} out of a {@link HTable}
*/ */
public static class HTableIncommon implements Incommon { public static class HTableIncommon implements Incommon {
final HTable table; final Table table;
/** /**
* @param table * @param table
*/ */
public HTableIncommon(final HTable table) { public HTableIncommon(final Table table) {
super(); super();
this.table = table; this.table = table;
} }

View File

@ -63,9 +63,11 @@ import org.apache.hadoop.hbase.client.HConnection;
import org.apache.hadoop.hbase.client.HConnectionManager; import org.apache.hadoop.hbase.client.HConnectionManager;
import org.apache.hadoop.hbase.client.HTable; import org.apache.hadoop.hbase.client.HTable;
import org.apache.hadoop.hbase.client.Put; 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.Result;
import org.apache.hadoop.hbase.client.ResultScanner; import org.apache.hadoop.hbase.client.ResultScanner;
import org.apache.hadoop.hbase.client.Scan; import org.apache.hadoop.hbase.client.Scan;
import org.apache.hadoop.hbase.client.Table;
import org.apache.hadoop.hbase.fs.HFileSystem; import org.apache.hadoop.hbase.fs.HFileSystem;
import org.apache.hadoop.hbase.io.compress.Compression; import org.apache.hadoop.hbase.io.compress.Compression;
import org.apache.hadoop.hbase.io.compress.Compression.Algorithm; import org.apache.hadoop.hbase.io.compress.Compression.Algorithm;
@ -899,7 +901,7 @@ public class HBaseTestingUtility extends HBaseCommonTestingUtility {
this.hbaseCluster = this.hbaseCluster =
new MiniHBaseCluster(c, numMasters, numSlaves, masterClass, regionserverClass); new MiniHBaseCluster(c, numMasters, numSlaves, masterClass, regionserverClass);
// Don't leave here till we've done a successful scan of the hbase:meta // Don't leave here till we've done a successful scan of the hbase:meta
HTable t = new HTable(c, TableName.META_TABLE_NAME); Table t = new HTable(c, TableName.META_TABLE_NAME);
ResultScanner s = t.getScanner(new Scan()); ResultScanner s = t.getScanner(new Scan());
while (s.next() != null) { while (s.next() != null) {
continue; continue;
@ -921,7 +923,7 @@ public class HBaseTestingUtility extends HBaseCommonTestingUtility {
public void restartHBaseCluster(int servers) throws IOException, InterruptedException { public void restartHBaseCluster(int servers) throws IOException, InterruptedException {
this.hbaseCluster = new MiniHBaseCluster(this.conf, servers); this.hbaseCluster = new MiniHBaseCluster(this.conf, servers);
// Don't leave here till we've done a successful scan of the hbase:meta // Don't leave here till we've done a successful scan of the hbase:meta
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()); ResultScanner s = t.getScanner(new Scan());
while (s.next() != null) { while (s.next() != null) {
// do nothing // do nothing
@ -1848,7 +1850,7 @@ public class HBaseTestingUtility extends HBaseCommonTestingUtility {
return rowCount; return rowCount;
} }
public void loadNumericRows(final HTable t, final byte[] f, int startRow, int endRow) throws IOException { public void loadNumericRows(final Table t, final byte[] f, int startRow, int endRow) throws IOException {
for (int i = startRow; i < endRow; i++) { for (int i = startRow; i < endRow; i++) {
byte[] data = Bytes.toBytes(String.valueOf(i)); byte[] data = Bytes.toBytes(String.valueOf(i));
Put put = new Put(data); Put put = new Put(data);
@ -1857,7 +1859,7 @@ public class HBaseTestingUtility extends HBaseCommonTestingUtility {
} }
} }
public void deleteNumericRows(final HTable t, final byte[] f, int startRow, int endRow) throws IOException { public void deleteNumericRows(final Table t, final byte[] f, int startRow, int endRow) throws IOException {
for (int i = startRow; i < endRow; i++) { for (int i = startRow; i < endRow; i++) {
byte[] data = Bytes.toBytes(String.valueOf(i)); byte[] data = Bytes.toBytes(String.valueOf(i));
Delete delete = new Delete(data); Delete delete = new Delete(data);
@ -1869,7 +1871,7 @@ public class HBaseTestingUtility extends HBaseCommonTestingUtility {
/** /**
* Return the number of rows in the given table. * 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(); Scan scan = new Scan();
ResultScanner results = table.getScanner(scan); ResultScanner results = table.getScanner(scan);
int count = 0; int count = 0;
@ -1880,7 +1882,7 @@ public class HBaseTestingUtility extends HBaseCommonTestingUtility {
return count; 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(); Scan scan = new Scan();
for (byte[] family: families) { for (byte[] family: families) {
scan.addFamily(family); scan.addFamily(family);
@ -1897,7 +1899,7 @@ public class HBaseTestingUtility extends HBaseCommonTestingUtility {
/** /**
* Return an md5 digest of the entire contents of a table. * 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(); Scan scan = new Scan();
ResultScanner results = table.getScanner(scan); ResultScanner results = table.getScanner(scan);
MessageDigest digest = MessageDigest.getInstance("MD5"); MessageDigest digest = MessageDigest.getInstance("MD5");
@ -2003,7 +2005,7 @@ public class HBaseTestingUtility extends HBaseCommonTestingUtility {
final byte[] columnFamily, byte [][] startKeys) final byte[] columnFamily, byte [][] startKeys)
throws IOException { throws IOException {
Arrays.sort(startKeys, Bytes.BYTES_COMPARATOR); 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(); HTableDescriptor htd = table.getTableDescriptor();
if(!htd.hasFamily(columnFamily)) { if(!htd.hasFamily(columnFamily)) {
HColumnDescriptor hcd = new HColumnDescriptor(columnFamily); HColumnDescriptor hcd = new HColumnDescriptor(columnFamily);
@ -2068,7 +2070,7 @@ public class HBaseTestingUtility extends HBaseCommonTestingUtility {
public List<HRegionInfo> createMultiRegionsInMeta(final Configuration conf, public List<HRegionInfo> createMultiRegionsInMeta(final Configuration conf,
final HTableDescriptor htd, byte [][] startKeys) final HTableDescriptor htd, byte [][] startKeys)
throws IOException { throws IOException {
HTable meta = new HTable(conf, TableName.META_TABLE_NAME); Table meta = new HTable(conf, TableName.META_TABLE_NAME);
Arrays.sort(startKeys, Bytes.BYTES_COMPARATOR); Arrays.sort(startKeys, Bytes.BYTES_COMPARATOR);
List<HRegionInfo> newRegions = new ArrayList<HRegionInfo>(startKeys.length); List<HRegionInfo> newRegions = new ArrayList<HRegionInfo>(startKeys.length);
// add custom ones // add custom ones
@ -2091,7 +2093,7 @@ public class HBaseTestingUtility extends HBaseCommonTestingUtility {
*/ */
public List<byte[]> getMetaTableRows() throws IOException { public List<byte[]> getMetaTableRows() throws IOException {
// TODO: Redo using MetaTableAccessor class // TODO: Redo using MetaTableAccessor class
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[]>(); List<byte[]> rows = new ArrayList<byte[]>();
ResultScanner s = t.getScanner(new Scan()); ResultScanner s = t.getScanner(new Scan());
for (Result result : s) { for (Result result : s) {
@ -2111,7 +2113,7 @@ public class HBaseTestingUtility extends HBaseCommonTestingUtility {
*/ */
public List<byte[]> getMetaTableRows(TableName tableName) throws IOException { public List<byte[]> getMetaTableRows(TableName tableName) throws IOException {
// TODO: Redo using MetaTableAccessor. // TODO: Redo using MetaTableAccessor.
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[]>(); List<byte[]> rows = new ArrayList<byte[]>();
ResultScanner s = t.getScanner(new Scan()); ResultScanner s = t.getScanner(new Scan());
for (Result result : s) { for (Result result : s) {
@ -2557,7 +2559,7 @@ public class HBaseTestingUtility extends HBaseCommonTestingUtility {
* @param table The table to find the region. * @param table The table to find the region.
* @throws IOException * @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); closeRegionByRow(Bytes.toBytes(row), table);
} }
@ -2568,7 +2570,7 @@ public class HBaseTestingUtility extends HBaseCommonTestingUtility {
* @param table The table to find the region. * @param table The table to find the region.
* @throws IOException * @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); HRegionLocation hrl = table.getRegionLocation(row);
closeRegion(hrl.getRegionInfo().getRegionName()); closeRegion(hrl.getRegionInfo().getRegionName());
} }
@ -2894,7 +2896,7 @@ public class HBaseTestingUtility extends HBaseCommonTestingUtility {
*/ */
public void waitUntilAllRegionsAssigned(final TableName tableName, final long timeout) public void waitUntilAllRegionsAssigned(final TableName tableName, final long timeout)
throws IOException { throws IOException {
final HTable meta = new HTable(getConfiguration(), TableName.META_TABLE_NAME); final Table meta = new HTable(getConfiguration(), TableName.META_TABLE_NAME);
try { try {
waitFor(timeout, 200, true, new Predicate<IOException>() { waitFor(timeout, 200, true, new Predicate<IOException>() {
@Override @Override
@ -3314,7 +3316,7 @@ public class HBaseTestingUtility extends HBaseCommonTestingUtility {
} }
public static int getMetaRSPort(Configuration conf) throws IOException { 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("")); HRegionLocation hloc = table.getRegionLocation(Bytes.toBytes(""));
table.close(); table.close();
return hloc.getPort(); return hloc.getPort();

View File

@ -58,11 +58,11 @@ import org.apache.hadoop.hbase.client.HBaseAdmin;
import org.apache.hadoop.hbase.client.HConnection; import org.apache.hadoop.hbase.client.HConnection;
import org.apache.hadoop.hbase.client.HConnectionManager; import org.apache.hadoop.hbase.client.HConnectionManager;
import org.apache.hadoop.hbase.client.HTable; import org.apache.hadoop.hbase.client.HTable;
import org.apache.hadoop.hbase.client.HTableInterface;
import org.apache.hadoop.hbase.client.Put; import org.apache.hadoop.hbase.client.Put;
import org.apache.hadoop.hbase.client.Result; import org.apache.hadoop.hbase.client.Result;
import org.apache.hadoop.hbase.client.ResultScanner; import org.apache.hadoop.hbase.client.ResultScanner;
import org.apache.hadoop.hbase.client.Scan; import org.apache.hadoop.hbase.client.Scan;
import org.apache.hadoop.hbase.client.Table;
import org.apache.hadoop.hbase.filter.BinaryComparator; import org.apache.hadoop.hbase.filter.BinaryComparator;
import org.apache.hadoop.hbase.filter.CompareFilter; import org.apache.hadoop.hbase.filter.CompareFilter;
import org.apache.hadoop.hbase.filter.Filter; import org.apache.hadoop.hbase.filter.Filter;
@ -892,7 +892,7 @@ public class PerformanceEvaluation extends Configured implements Tool {
private final Sampler<?> traceSampler; private final Sampler<?> traceSampler;
private final SpanReceiverHost receiverHost; private final SpanReceiverHost receiverHost;
protected HConnection connection; protected HConnection connection;
protected HTableInterface table; protected Table table;
private String testName; private String testName;
private Histogram latency; private Histogram latency;
@ -1531,7 +1531,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 // 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. // the TestOptions introspection for us and dump the output in a readable format.
LOG.info(cmd.getSimpleName() + " test run options=" + MAPPER.writeValueAsString(opts)); LOG.info(cmd.getSimpleName() + " test run options=" + MAPPER.writeValueAsString(opts));
HBaseAdmin admin = null; Admin admin = null;
try { try {
admin = new HBaseAdmin(getConf()); admin = new HBaseAdmin(getConf());
checkTable(admin, opts); 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.Result;
import org.apache.hadoop.hbase.client.ResultScanner; import org.apache.hadoop.hbase.client.ResultScanner;
import org.apache.hadoop.hbase.client.Scan; import org.apache.hadoop.hbase.client.Scan;
import org.apache.hadoop.hbase.client.Table;
import org.apache.hadoop.hbase.client.TableSnapshotScanner; import org.apache.hadoop.hbase.client.TableSnapshotScanner;
import org.apache.hadoop.hbase.client.metrics.ScanMetrics; import org.apache.hadoop.hbase.client.metrics.ScanMetrics;
import org.apache.hadoop.hbase.io.ImmutableBytesWritable; import org.apache.hadoop.hbase.io.ImmutableBytesWritable;
@ -146,7 +147,7 @@ public class ScanPerformanceEvaluation extends AbstractHBaseTool {
Stopwatch scanTimer = new Stopwatch(); Stopwatch scanTimer = new Stopwatch();
tableOpenTimer.start(); tableOpenTimer.start();
HTable table = new HTable(getConf(), TableName.valueOf(tablename)); Table table = new HTable(getConf(), TableName.valueOf(tablename));
tableOpenTimer.stop(); tableOpenTimer.stop();
Scan scan = getScan(); 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.Result;
import org.apache.hadoop.hbase.client.ResultScanner; import org.apache.hadoop.hbase.client.ResultScanner;
import org.apache.hadoop.hbase.client.Scan; import org.apache.hadoop.hbase.client.Scan;
import org.apache.hadoop.hbase.client.Table;
import org.apache.hadoop.hbase.regionserver.ConstantSizeRegionSplitPolicy; import org.apache.hadoop.hbase.regionserver.ConstantSizeRegionSplitPolicy;
import org.apache.hadoop.hbase.util.Bytes; import org.apache.hadoop.hbase.util.Bytes;
import org.apache.hadoop.util.StringUtils; import org.apache.hadoop.util.StringUtils;
@ -97,7 +98,7 @@ public class TestAcidGuarantees implements Tool {
byte data[] = new byte[10]; byte data[] = new byte[10];
byte targetRows[][]; byte targetRows[][];
byte targetFamilies[][]; byte targetFamilies[][];
HTable table; Table table;
AtomicLong numWritten = new AtomicLong(); AtomicLong numWritten = new AtomicLong();
public AtomicityWriter(TestContext ctx, byte targetRows[][], public AtomicityWriter(TestContext ctx, byte targetRows[][],
@ -131,7 +132,7 @@ public class TestAcidGuarantees implements Tool {
public static class AtomicGetReader extends RepeatingTestThread { public static class AtomicGetReader extends RepeatingTestThread {
byte targetRow[]; byte targetRow[];
byte targetFamilies[][]; byte targetFamilies[][];
HTable table; Table table;
int numVerified = 0; int numVerified = 0;
AtomicLong numRead = new AtomicLong(); AtomicLong numRead = new AtomicLong();
@ -189,7 +190,7 @@ public class TestAcidGuarantees implements Tool {
*/ */
public static class AtomicScanReader extends RepeatingTestThread { public static class AtomicScanReader extends RepeatingTestThread {
byte targetFamilies[][]; byte targetFamilies[][];
HTable table; Table table;
AtomicLong numScans = new AtomicLong(); AtomicLong numScans = new AtomicLong();
AtomicLong numRowsScanned = 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.FileSystem;
import org.apache.hadoop.fs.Path; import org.apache.hadoop.fs.Path;
import org.apache.hadoop.hbase.client.Get; 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.Put;
import org.apache.hadoop.hbase.client.Result; import org.apache.hadoop.hbase.client.Result;
import org.apache.hadoop.hbase.client.Table;
import org.apache.hadoop.hbase.util.Bytes; import org.apache.hadoop.hbase.util.Bytes;
import org.apache.hadoop.hbase.zookeeper.MiniZooKeeperCluster; import org.apache.hadoop.hbase.zookeeper.MiniZooKeeperCluster;
import org.apache.hadoop.hdfs.MiniDFSCluster; import org.apache.hadoop.hdfs.MiniDFSCluster;
@ -87,8 +87,8 @@ public class TestHBaseTestingUtility {
final byte[] QUAL_NAME = Bytes.toBytes("qual"); final byte[] QUAL_NAME = Bytes.toBytes("qual");
final byte[] VALUE = Bytes.toBytes("value"); final byte[] VALUE = Bytes.toBytes("value");
HTable table1 = htu1.createTable(TABLE_NAME, FAM_NAME); Table table1 = htu1.createTable(TABLE_NAME, FAM_NAME);
HTable table2 = htu2.createTable(TABLE_NAME, FAM_NAME); Table table2 = htu2.createTable(TABLE_NAME, FAM_NAME);
Put put = new Put(ROW); Put put = new Put(ROW);
put.add(FAM_NAME, QUAL_NAME, VALUE); 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.HConnectionManager;
import org.apache.hadoop.hbase.client.HTable; import org.apache.hadoop.hbase.client.HTable;
import org.apache.hadoop.hbase.client.Result; import org.apache.hadoop.hbase.client.Result;
import org.apache.hadoop.hbase.client.Table;
import org.apache.hadoop.hbase.util.Bytes; import org.apache.hadoop.hbase.util.Bytes;
import org.apache.hadoop.hbase.util.Pair; import org.apache.hadoop.hbase.util.Pair;
import org.junit.AfterClass; import org.junit.AfterClass;
@ -332,7 +333,7 @@ public class TestMetaTableAccessor {
long seqNum100 = random.nextLong(); long seqNum100 = random.nextLong();
HTable meta = MetaTableAccessor.getMetaHTable(hConnection); Table meta = MetaTableAccessor.getMetaHTable(hConnection);
try { try {
MetaTableAccessor.updateRegionLocation(hConnection, primary, serverName0, seqNum0); MetaTableAccessor.updateRegionLocation(hConnection, primary, serverName0, seqNum0);
@ -359,7 +360,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 { long seqNum, int replicaId, boolean checkSeqNum) throws IOException {
Get get = new Get(row); Get get = new Get(row);
Result result = meta.get(get); 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.FlushCache;
import org.apache.hadoop.hbase.HBaseTestCase.HTableIncommon; import org.apache.hadoop.hbase.HBaseTestCase.HTableIncommon;
import org.apache.hadoop.hbase.HBaseTestCase.Incommon; 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.Get;
import org.apache.hadoop.hbase.client.HBaseAdmin; import org.apache.hadoop.hbase.client.HBaseAdmin;
import org.apache.hadoop.hbase.client.HTable; import org.apache.hadoop.hbase.client.HTable;
@ -40,6 +41,7 @@ import org.apache.hadoop.hbase.client.Put;
import org.apache.hadoop.hbase.client.Result; import org.apache.hadoop.hbase.client.Result;
import org.apache.hadoop.hbase.client.ResultScanner; import org.apache.hadoop.hbase.client.ResultScanner;
import org.apache.hadoop.hbase.client.Scan; import org.apache.hadoop.hbase.client.Scan;
import org.apache.hadoop.hbase.client.Table;
import org.apache.hadoop.hbase.util.Bytes; import org.apache.hadoop.hbase.util.Bytes;
import org.junit.After; import org.junit.After;
import org.junit.AfterClass; import org.junit.AfterClass;
@ -56,7 +58,7 @@ import org.junit.experimental.categories.Category;
public class TestMultiVersions { public class TestMultiVersions {
private static final Log LOG = LogFactory.getLog(TestMultiVersions.class); private static final Log LOG = LogFactory.getLog(TestMultiVersions.class);
private static final HBaseTestingUtility UTIL = new HBaseTestingUtility(); private static final HBaseTestingUtility UTIL = new HBaseTestingUtility();
private HBaseAdmin admin; private Admin admin;
private static final int NUM_SLAVES = 3; private static final int NUM_SLAVES = 3;
@ -97,7 +99,7 @@ public class TestMultiVersions {
hcd.setMaxVersions(3); hcd.setMaxVersions(3);
desc.addFamily(hcd); desc.addFamily(hcd);
this.admin.createTable(desc); 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 // TODO: Remove these deprecated classes or pull them in here if this is
// only test using them. // only test using them.
Incommon incommon = new HTableIncommon(table); Incommon incommon = new HTableIncommon(table);
@ -140,7 +142,7 @@ public class TestMultiVersions {
this.admin.createTable(desc); this.admin.createTable(desc);
Put put = new Put(row, timestamp1); Put put = new Put(row, timestamp1);
put.add(contents, contents, value1); put.add(contents, contents, value1);
HTable table = new HTable(UTIL.getConfiguration(), tableName); Table table = new HTable(UTIL.getConfiguration(), tableName);
table.put(put); table.put(put);
// Shut down and restart the HBase cluster // Shut down and restart the HBase cluster
table.close(); 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.Get;
import org.apache.hadoop.hbase.client.HTable; import org.apache.hadoop.hbase.client.HTable;
import org.apache.hadoop.hbase.client.Put; import org.apache.hadoop.hbase.client.Put;
import org.apache.hadoop.hbase.client.Table;
import org.apache.hadoop.hbase.master.HMaster; import org.apache.hadoop.hbase.master.HMaster;
import org.apache.hadoop.hbase.util.Bytes; import org.apache.hadoop.hbase.util.Bytes;
import org.apache.hadoop.hbase.util.FSUtils; import org.apache.hadoop.hbase.util.FSUtils;
@ -249,7 +250,7 @@ public class TestNamespace {
} }
//sanity check try to write and read from table //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")); Put p = new Put(Bytes.toBytes("row1"));
p.add(Bytes.toBytes("my_cf"),Bytes.toBytes("my_col"),Bytes.toBytes("value1")); p.add(Bytes.toBytes("my_cf"),Bytes.toBytes("my_col"),Bytes.toBytes("value1"));
table.put(p); table.put(p);

View File

@ -29,8 +29,10 @@ import java.util.List;
import org.apache.commons.logging.Log; import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory; import org.apache.commons.logging.LogFactory;
import org.apache.hadoop.hbase.client.Admin;
import org.apache.hadoop.hbase.client.HBaseAdmin; import org.apache.hadoop.hbase.client.HBaseAdmin;
import org.apache.hadoop.hbase.client.HTable; import org.apache.hadoop.hbase.client.HTable;
import org.apache.hadoop.hbase.client.RegionLocator;
import org.apache.hadoop.hbase.master.RegionStates; import org.apache.hadoop.hbase.master.RegionStates;
import org.apache.hadoop.hbase.protobuf.ProtobufUtil; import org.apache.hadoop.hbase.protobuf.ProtobufUtil;
import org.apache.hadoop.hbase.regionserver.HRegionServer; import org.apache.hadoop.hbase.regionserver.HRegionServer;
@ -63,7 +65,7 @@ public class TestRegionRebalancing {
private static final byte[] FAMILY_NAME = Bytes.toBytes("col"); private static final byte[] FAMILY_NAME = Bytes.toBytes("col");
public static final Log LOG = LogFactory.getLog(TestRegionRebalancing.class); public static final Log LOG = LogFactory.getLog(TestRegionRebalancing.class);
private final HBaseTestingUtility UTIL = new HBaseTestingUtility(); private final HBaseTestingUtility UTIL = new HBaseTestingUtility();
private HTable table; private RegionLocator table;
private HTableDescriptor desc; private HTableDescriptor desc;
private String balancerName; private String balancerName;
@ -94,7 +96,7 @@ public class TestRegionRebalancing {
@Test (timeout=300000) @Test (timeout=300000)
public void testRebalanceOnRegionServerNumberChange() public void testRebalanceOnRegionServerNumberChange()
throws IOException, InterruptedException { throws IOException, InterruptedException {
HBaseAdmin admin = new HBaseAdmin(UTIL.getConfiguration()); Admin admin = new HBaseAdmin(UTIL.getConfiguration());
admin.createTable(this.desc, Arrays.copyOfRange(HBaseTestingUtility.KEYS, admin.createTable(this.desc, Arrays.copyOfRange(HBaseTestingUtility.KEYS,
1, HBaseTestingUtility.KEYS.length)); 1, HBaseTestingUtility.KEYS.length));
this.table = new HTable(UTIL.getConfiguration(), this.desc.getTableName()); 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.Log;
import org.apache.commons.logging.LogFactory; import org.apache.commons.logging.LogFactory;
import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.hbase.client.Admin;
import org.apache.hadoop.hbase.client.Get; import org.apache.hadoop.hbase.client.Get;
import org.apache.hadoop.hbase.client.HBaseAdmin; import org.apache.hadoop.hbase.client.HBaseAdmin;
import org.apache.hadoop.hbase.client.HConnection; import org.apache.hadoop.hbase.client.HConnection;
@ -43,6 +44,7 @@ import org.apache.hadoop.hbase.client.Put;
import org.apache.hadoop.hbase.client.Result; import org.apache.hadoop.hbase.client.Result;
import org.apache.hadoop.hbase.client.ResultScanner; import org.apache.hadoop.hbase.client.ResultScanner;
import org.apache.hadoop.hbase.client.Scan; import org.apache.hadoop.hbase.client.Scan;
import org.apache.hadoop.hbase.client.Table;
import org.apache.hadoop.hbase.master.HMaster; import org.apache.hadoop.hbase.master.HMaster;
import org.apache.hadoop.hbase.master.LoadBalancer; import org.apache.hadoop.hbase.master.LoadBalancer;
import org.apache.hadoop.hbase.master.balancer.SimpleLoadBalancer; import org.apache.hadoop.hbase.master.balancer.SimpleLoadBalancer;
@ -250,14 +252,14 @@ public class TestZooKeeper {
HColumnDescriptor family = new HColumnDescriptor("fam"); HColumnDescriptor family = new HColumnDescriptor("fam");
desc.addFamily(family); desc.addFamily(family);
LOG.info("Creating table " + tableName); LOG.info("Creating table " + tableName);
HBaseAdmin admin = new HBaseAdmin(TEST_UTIL.getConfiguration()); Admin admin = new HBaseAdmin(TEST_UTIL.getConfiguration());
try { try {
admin.createTable(desc); admin.createTable(desc);
} finally { } finally {
admin.close(); admin.close();
} }
HTable table = Table table =
new HTable(new Configuration(TEST_UTIL.getConfiguration()), tableName); new HTable(new Configuration(TEST_UTIL.getConfiguration()), tableName);
Put put = new Put(Bytes.toBytes("testrow")); Put put = new Put(Bytes.toBytes("testrow"));
put.add(Bytes.toBytes("fam"), put.add(Bytes.toBytes("fam"),
@ -270,11 +272,11 @@ public class TestZooKeeper {
@Test @Test
public void testMultipleZK() public void testMultipleZK()
throws IOException, NoSuchMethodException, InvocationTargetException, IllegalAccessException { throws IOException, NoSuchMethodException, InvocationTargetException, IllegalAccessException {
HTable localMeta = Table localMeta =
new HTable(new Configuration(TEST_UTIL.getConfiguration()), TableName.META_TABLE_NAME); new HTable(new Configuration(TEST_UTIL.getConfiguration()), TableName.META_TABLE_NAME);
Configuration otherConf = new Configuration(TEST_UTIL.getConfiguration()); Configuration otherConf = new Configuration(TEST_UTIL.getConfiguration());
otherConf.set(HConstants.ZOOKEEPER_QUORUM, "127.0.0.1"); otherConf.set(HConstants.ZOOKEEPER_QUORUM, "127.0.0.1");
HTable ipMeta = new HTable(otherConf, TableName.META_TABLE_NAME); Table ipMeta = new HTable(otherConf, TableName.META_TABLE_NAME);
// dummy, just to open the connection // dummy, just to open the connection
final byte [] row = new byte [] {'r'}; final byte [] row = new byte [] {'r'};
@ -491,7 +493,7 @@ public class TestZooKeeper {
ZooKeeperWatcher zkw = m.getZooKeeper(); ZooKeeperWatcher zkw = m.getZooKeeper();
int expectedNumOfListeners = zkw.getNumberOfListeners(); int expectedNumOfListeners = zkw.getNumberOfListeners();
// now the cluster is up. So assign some regions. // now the cluster is up. So assign some regions.
HBaseAdmin admin = new HBaseAdmin(TEST_UTIL.getConfiguration()); Admin admin = new HBaseAdmin(TEST_UTIL.getConfiguration());
try { try {
byte[][] SPLIT_KEYS = new byte[][] { Bytes.toBytes("a"), Bytes.toBytes("b"), byte[][] SPLIT_KEYS = new byte[][] { Bytes.toBytes("a"), Bytes.toBytes("b"),
Bytes.toBytes("c"), Bytes.toBytes("d"), Bytes.toBytes("e"), Bytes.toBytes("f"), Bytes.toBytes("c"), Bytes.toBytes("d"), Bytes.toBytes("e"), Bytes.toBytes("f"),
@ -530,8 +532,8 @@ public class TestZooKeeper {
cluster.startRegionServer(); cluster.startRegionServer();
HMaster m = cluster.getMaster(); HMaster m = cluster.getMaster();
// now the cluster is up. So assign some regions. // now the cluster is up. So assign some regions.
HBaseAdmin admin = new HBaseAdmin(TEST_UTIL.getConfiguration()); Admin admin = new HBaseAdmin(TEST_UTIL.getConfiguration());
HTable table = null; Table table = null;
try { try {
byte[][] SPLIT_KEYS = new byte[][] { Bytes.toBytes("1"), Bytes.toBytes("2"), byte[][] SPLIT_KEYS = new byte[][] { Bytes.toBytes("1"), Bytes.toBytes("2"),
Bytes.toBytes("3"), Bytes.toBytes("4"), Bytes.toBytes("5") }; 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 [] qualifier = Bytes.toBytes("qualifier");
final byte [] value = Bytes.toBytes("value"); final byte [] value = Bytes.toBytes("value");
final TableName table = TableName.valueOf("testDisableAndEnableTable"); 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 put = new Put(row);
put.add(HConstants.CATALOG_FAMILY, qualifier, value); put.add(HConstants.CATALOG_FAMILY, qualifier, value);
ht.put(put); ht.put(put);
@ -291,8 +291,8 @@ public class TestAdmin {
final byte [] value = Bytes.toBytes("value"); final byte [] value = Bytes.toBytes("value");
final byte [] table1 = Bytes.toBytes("testDisableAndEnableTable1"); final byte [] table1 = Bytes.toBytes("testDisableAndEnableTable1");
final byte [] table2 = Bytes.toBytes("testDisableAndEnableTable2"); final byte [] table2 = Bytes.toBytes("testDisableAndEnableTable2");
HTable ht1 = TEST_UTIL.createTable(table1, HConstants.CATALOG_FAMILY); Table ht1 = TEST_UTIL.createTable(table1, HConstants.CATALOG_FAMILY);
HTable ht2 = TEST_UTIL.createTable(table2, HConstants.CATALOG_FAMILY); Table ht2 = TEST_UTIL.createTable(table2, HConstants.CATALOG_FAMILY);
Put put = new Put(row); Put put = new Put(row);
put.add(HConstants.CATALOG_FAMILY, qualifier, value); put.add(HConstants.CATALOG_FAMILY, qualifier, value);
ht1.put(put); ht1.put(put);
@ -400,7 +400,7 @@ public class TestAdmin {
htd.addFamily(fam2); htd.addFamily(fam2);
htd.addFamily(fam3); htd.addFamily(fam3);
this.admin.createTable(htd); this.admin.createTable(htd);
HTable table = new HTable(TEST_UTIL.getConfiguration(), "myTestTable"); Table table = new HTable(TEST_UTIL.getConfiguration(), "myTestTable");
HTableDescriptor confirmedHtd = table.getTableDescriptor(); HTableDescriptor confirmedHtd = table.getTableDescriptor();
assertEquals(htd.compareTo(confirmedHtd), 0); assertEquals(htd.compareTo(confirmedHtd), 0);
table.close(); table.close();
@ -816,7 +816,7 @@ public class TestAdmin {
TableName TABLE_4 = TableName.valueOf(tableName.getNameAsString() + "_4"); TableName TABLE_4 = TableName.valueOf(tableName.getNameAsString() + "_4");
desc = new HTableDescriptor(TABLE_4); desc = new HTableDescriptor(TABLE_4);
desc.addFamily(new HColumnDescriptor(HConstants.CATALOG_FAMILY)); desc.addFamily(new HColumnDescriptor(HConstants.CATALOG_FAMILY));
HBaseAdmin ladmin = new HBaseAdmin(TEST_UTIL.getConfiguration()); Admin ladmin = new HBaseAdmin(TEST_UTIL.getConfiguration());
try { try {
ladmin.createTable(desc, splitKeys); ladmin.createTable(desc, splitKeys);
assertTrue("Should not be able to create this table because of " + assertTrue("Should not be able to create this table because of " +
@ -1238,7 +1238,7 @@ public class TestAdmin {
// Use 80 bit numbers to make sure we aren't limited // Use 80 bit numbers to make sure we aren't limited
byte [] startKey = { 1, 1, 1, 1, 1, 1, 1, 1, 1, 1 }; byte [] startKey = { 1, 1, 1, 1, 1, 1, 1, 1, 1, 1 };
byte [] endKey = { 9, 9, 9, 9, 9, 9, 9, 9, 9, 9 }; byte [] endKey = { 9, 9, 9, 9, 9, 9, 9, 9, 9, 9 };
HBaseAdmin hbaseadmin = new HBaseAdmin(TEST_UTIL.getConfiguration()); Admin hbaseadmin = new HBaseAdmin(TEST_UTIL.getConfiguration());
HTableDescriptor htd = new HTableDescriptor(TableName.valueOf(name)); HTableDescriptor htd = new HTableDescriptor(TableName.valueOf(name));
htd.addFamily(new HColumnDescriptor(HConstants.CATALOG_FAMILY)); htd.addFamily(new HColumnDescriptor(HConstants.CATALOG_FAMILY));
hbaseadmin.createTable(htd, startKey, endKey, expectedRegions); hbaseadmin.createTable(htd, startKey, endKey, expectedRegions);
@ -1255,7 +1255,7 @@ public class TestAdmin {
@Test (timeout=300000) @Test (timeout=300000)
public void testReadOnlyTable() throws Exception { public void testReadOnlyTable() throws Exception {
byte [] name = Bytes.toBytes("testReadOnlyTable"); 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"); byte[] value = Bytes.toBytes("somedata");
// This used to use an empty row... That must have been a bug // This used to use an empty row... That must have been a bug
Put put = new Put(value); Put put = new Put(value);
@ -1324,7 +1324,7 @@ public class TestAdmin {
@Test (expected=TableNotDisabledException.class, timeout=300000) @Test (expected=TableNotDisabledException.class, timeout=300000)
public void testTableNotDisabledExceptionWithATable() throws IOException { public void testTableNotDisabledExceptionWithATable() throws IOException {
final TableName name = TableName.valueOf("testTableNotDisabledExceptionWithATable"); 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 { try {
this.admin.enableTable(name); this.admin.enableTable(name);
}finally { }finally {
@ -1338,7 +1338,7 @@ public class TestAdmin {
*/ */
@Test (expected=TableNotFoundException.class, timeout=300000) @Test (expected=TableNotFoundException.class, timeout=300000)
public void testTableNotFoundExceptionWithoutAnyTables() throws IOException { public void testTableNotFoundExceptionWithoutAnyTables() throws IOException {
HTable ht = Table ht =
new HTable(TEST_UTIL.getConfiguration(),"testTableNotFoundExceptionWithoutAnyTables"); new HTable(TEST_UTIL.getConfiguration(),"testTableNotFoundExceptionWithoutAnyTables");
ht.get(new Get("e".getBytes())); ht.get(new Get("e".getBytes()));
} }
@ -1659,7 +1659,7 @@ public class TestAdmin {
HTableDescriptor desc = new HTableDescriptor(TableName.valueOf(tableName)); HTableDescriptor desc = new HTableDescriptor(TableName.valueOf(tableName));
desc.addFamily(new HColumnDescriptor(HConstants.CATALOG_FAMILY)); desc.addFamily(new HColumnDescriptor(HConstants.CATALOG_FAMILY));
admin.createTable(desc); 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)); HRegionServer regionServer = TEST_UTIL.getRSForFirstRegionInTable(Bytes.toBytes(tableName));
for (int i = 1; i <= 256; i++) { // 256 writes should cause 8 log rolls for (int i = 1; i <= 256; i++) { // 256 writes should cause 8 log rolls

View File

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

View File

@ -84,7 +84,7 @@ public class TestClientScannerRPCTimeout {
@Test @Test
public void testScannerNextRPCTimesout() throws Exception { public void testScannerNextRPCTimesout() throws Exception {
final byte[] TABLE_NAME = Bytes.toBytes("testScannerNextRPCTimesout"); 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[] r1 = Bytes.toBytes("row-1");
byte[] r2 = Bytes.toBytes("row-2"); byte[] r2 = Bytes.toBytes("row-2");
byte[] r3 = Bytes.toBytes("row-3"); byte[] r3 = Bytes.toBytes("row-3");
@ -123,7 +123,7 @@ public class TestClientScannerRPCTimeout {
RSRpcServicesWithScanTimeout.tryNumber <= CLIENT_RETRIES_NUMBER); 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 put = new Put(rowkey);
put.add(FAMILY, QUALIFIER, VALUE); put.add(FAMILY, QUALIFIER, VALUE);
ht.put(put); ht.put(put);

View File

@ -87,7 +87,7 @@ public class TestClientTimeouts {
// Ensure the HBaseAdmin uses a new connection by changing Configuration. // Ensure the HBaseAdmin uses a new connection by changing Configuration.
Configuration conf = HBaseConfiguration.create(TEST_UTIL.getConfiguration()); Configuration conf = HBaseConfiguration.create(TEST_UTIL.getConfiguration());
conf.set(HConstants.HBASE_CLIENT_INSTANCE_ID, String.valueOf(-1)); conf.set(HConstants.HBASE_CLIENT_INSTANCE_ID, String.valueOf(-1));
HBaseAdmin admin = null; Admin admin = null;
try { try {
admin = new HBaseAdmin(conf); admin = new HBaseAdmin(conf);
HConnection connection = admin.getConnection(); HConnection connection = admin.getConnection();

View File

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

View File

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

View File

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

View File

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

View File

@ -21,6 +21,10 @@ package org.apache.hadoop.hbase.client;
import java.io.IOException; import java.io.IOException;
import org.apache.hadoop.hbase.*; import org.apache.hadoop.hbase.*;
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.Table;
import org.apache.hadoop.hbase.util.Bytes; import org.apache.hadoop.hbase.util.Bytes;
import org.apache.hadoop.hbase.util.PoolMap.PoolType; import org.apache.hadoop.hbase.util.PoolMap.PoolType;
import org.junit.*; import org.junit.*;
@ -60,14 +64,14 @@ public class TestHTablePool {
String tableName = Bytes.toString(TABLENAME); String tableName = Bytes.toString(TABLENAME);
// Request a table from an empty pool // Request a table from an empty pool
HTableInterface table = pool.getTable(tableName); Table table = pool.getTable(tableName);
Assert.assertNotNull(table); Assert.assertNotNull(table);
// Close table (returns table to the pool) // Close table (returns table to the pool)
table.close(); table.close();
// Request a table of the same name // Request a table of the same name
HTableInterface sameTable = pool.getTable(tableName); Table sameTable = pool.getTable(tableName);
Assert.assertSame( Assert.assertSame(
((HTablePool.PooledHTable) table).getWrappedTable(), ((HTablePool.PooledHTable) table).getWrappedTable(),
((HTablePool.PooledHTable) sameTable).getWrappedTable()); ((HTablePool.PooledHTable) sameTable).getWrappedTable());
@ -79,14 +83,14 @@ public class TestHTablePool {
Integer.MAX_VALUE, getPoolType()); Integer.MAX_VALUE, getPoolType());
// Request a table from an empty pool // Request a table from an empty pool
HTableInterface table = pool.getTable(TABLENAME); Table table = pool.getTable(TABLENAME);
Assert.assertNotNull(table); Assert.assertNotNull(table);
// Close table (returns table to the pool) // Close table (returns table to the pool)
table.close(); table.close();
// Request a table of the same name // Request a table of the same name
HTableInterface sameTable = pool.getTable(TABLENAME); Table sameTable = pool.getTable(TABLENAME);
Assert.assertSame( Assert.assertSame(
((HTablePool.PooledHTable) table).getWrappedTable(), ((HTablePool.PooledHTable) table).getWrappedTable(),
((HTablePool.PooledHTable) sameTable).getWrappedTable()); ((HTablePool.PooledHTable) sameTable).getWrappedTable());
@ -104,8 +108,8 @@ public class TestHTablePool {
TEST_UTIL.createTable(otherTable, HConstants.CATALOG_FAMILY); TEST_UTIL.createTable(otherTable, HConstants.CATALOG_FAMILY);
// Request a table from an empty pool // Request a table from an empty pool
HTableInterface table1 = pool.getTable(TABLENAME); Table table1 = pool.getTable(TABLENAME);
HTableInterface table2 = pool.getTable(otherTable); Table table2 = pool.getTable(otherTable);
Assert.assertNotNull(table2); Assert.assertNotNull(table2);
// Close tables (returns tables to the pool) // Close tables (returns tables to the pool)
@ -113,8 +117,8 @@ public class TestHTablePool {
table2.close(); table2.close();
// Request tables of the same names // Request tables of the same names
HTableInterface sameTable1 = pool.getTable(TABLENAME); Table sameTable1 = pool.getTable(TABLENAME);
HTableInterface sameTable2 = pool.getTable(otherTable); Table sameTable2 = pool.getTable(otherTable);
Assert.assertSame( Assert.assertSame(
((HTablePool.PooledHTable) table1).getWrappedTable(), ((HTablePool.PooledHTable) table1).getWrappedTable(),
((HTablePool.PooledHTable) sameTable1).getWrappedTable()); ((HTablePool.PooledHTable) sameTable1).getWrappedTable());
@ -129,7 +133,7 @@ public class TestHTablePool {
String tableName = Bytes.toString(TABLENAME);// Request a table from String tableName = Bytes.toString(TABLENAME);// Request a table from
// an // an
// empty pool // empty pool
HTableInterface table = pool.getTable(tableName); Table table = pool.getTable(tableName);
// Test if proxy implementation is returned // Test if proxy implementation is returned
Assert.assertTrue(table instanceof HTablePool.PooledHTable); Assert.assertTrue(table instanceof HTablePool.PooledHTable);
@ -150,7 +154,7 @@ public class TestHTablePool {
pool.putTable(table); pool.putTable(table);
// Request a table of the same name // Request a table of the same name
HTableInterface sameTable = pool.getTable(tableName); Table sameTable = pool.getTable(tableName);
// test no proxy over proxy created // test no proxy over proxy created
Assert.assertSame(((HTablePool.PooledHTable) table).getWrappedTable(), Assert.assertSame(((HTablePool.PooledHTable) table).getWrappedTable(),
@ -166,7 +170,7 @@ public class TestHTablePool {
// empty pool // empty pool
// get table will return proxy implementation // get table will return proxy implementation
final HTableInterface table = pool.getTable(tableName); final Table table = pool.getTable(tableName);
HTableInterface alienTable = new HTable(TEST_UTIL.getConfiguration(), HTableInterface alienTable = new HTable(TEST_UTIL.getConfiguration(),
TABLENAME) { TABLENAME) {
// implementation doesn't matter as long the table is not from // implementation doesn't matter as long the table is not from
@ -188,7 +192,7 @@ public class TestHTablePool {
String tableName = Bytes.toString(TABLENAME); String tableName = Bytes.toString(TABLENAME);
// Request a table from an empty pool // Request a table from an empty pool
HTableInterface table = pool.getTable(tableName); Table table = pool.getTable(tableName);
Assert.assertNotNull(table); Assert.assertNotNull(table);
Assert.assertTrue(((HTablePool.PooledHTable) table).isOpen()); Assert.assertTrue(((HTablePool.PooledHTable) table).isOpen());
// Close table (returns table to the pool) // Close table (returns table to the pool)
@ -223,9 +227,9 @@ public class TestHTablePool {
getPoolType()); getPoolType());
// Request tables from an empty pool // Request tables from an empty pool
HTableInterface table1 = pool.getTable(TABLENAME); Table table1 = pool.getTable(TABLENAME);
HTableInterface table2 = pool.getTable(TABLENAME); Table table2 = pool.getTable(TABLENAME);
HTableInterface table3 = pool.getTable(TABLENAME); Table table3 = pool.getTable(TABLENAME);
// Close tables (returns tables to the pool) // Close tables (returns tables to the pool)
table1.close(); table1.close();
@ -234,9 +238,9 @@ public class TestHTablePool {
table3.close(); table3.close();
// Request tables of the same name // Request tables of the same name
HTableInterface sameTable1 = pool.getTable(TABLENAME); Table sameTable1 = pool.getTable(TABLENAME);
HTableInterface sameTable2 = pool.getTable(TABLENAME); Table sameTable2 = pool.getTable(TABLENAME);
HTableInterface sameTable3 = pool.getTable(TABLENAME); Table sameTable3 = pool.getTable(TABLENAME);
Assert.assertSame( Assert.assertSame(
((HTablePool.PooledHTable) table1).getWrappedTable(), ((HTablePool.PooledHTable) table1).getWrappedTable(),
((HTablePool.PooledHTable) sameTable1).getWrappedTable()); ((HTablePool.PooledHTable) sameTable1).getWrappedTable());
@ -264,7 +268,7 @@ public class TestHTablePool {
admin.createTable(tableDescriptor); admin.createTable(tableDescriptor);
// Request tables from an empty pool // Request tables from an empty pool
HTableInterface[] tables = new HTableInterface[4]; Table[] tables = new Table[4];
for (int i = 0; i < 4; ++i) { for (int i = 0; i < 4; ++i) {
tables[i] = pool.getTable(TABLENAME); tables[i] = pool.getTable(TABLENAME);
} }
@ -298,9 +302,9 @@ public class TestHTablePool {
getPoolType()); getPoolType());
// Request tables from an empty pool // Request tables from an empty pool
HTableInterface table1 = pool.getTable(TABLENAME); Table table1 = pool.getTable(TABLENAME);
HTableInterface table2 = pool.getTable(TABLENAME); Table table2 = pool.getTable(TABLENAME);
HTableInterface table3 = pool.getTable(TABLENAME); Table table3 = pool.getTable(TABLENAME);
// Close tables (returns tables to the pool) // Close tables (returns tables to the pool)
table1.close(); table1.close();
@ -310,9 +314,9 @@ public class TestHTablePool {
table3.close(); table3.close();
// Request tables of the same name // Request tables of the same name
HTableInterface sameTable1 = pool.getTable(TABLENAME); Table sameTable1 = pool.getTable(TABLENAME);
HTableInterface sameTable2 = pool.getTable(TABLENAME); Table sameTable2 = pool.getTable(TABLENAME);
HTableInterface sameTable3 = pool.getTable(TABLENAME); Table sameTable3 = pool.getTable(TABLENAME);
Assert.assertSame( Assert.assertSame(
((HTablePool.PooledHTable) table3).getWrappedTable(), ((HTablePool.PooledHTable) table3).getWrappedTable(),
((HTablePool.PooledHTable) sameTable1).getWrappedTable()); ((HTablePool.PooledHTable) sameTable1).getWrappedTable());
@ -340,7 +344,7 @@ public class TestHTablePool {
admin.createTable(tableDescriptor); admin.createTable(tableDescriptor);
// Request tables from an empty pool // Request tables from an empty pool
HTableInterface[] tables = new HTableInterface[4]; Table[] tables = new Table[4];
for (int i = 0; i < 4; ++i) { for (int i = 0; i < 4; ++i) {
tables[i] = pool.getTable(TABLENAME); tables[i] = pool.getTable(TABLENAME);
} }

View File

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

View File

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

View File

@ -52,7 +52,7 @@ public class TestPutWithDelete {
final TableName tableName = TableName.valueOf("TestPutWithDelete"); final TableName tableName = TableName.valueOf("TestPutWithDelete");
final byte[] rowKey = Bytes.toBytes("12345"); final byte[] rowKey = Bytes.toBytes("12345");
final byte[] family = Bytes.toBytes("cf"); 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); TEST_UTIL.waitTableAvailable(tableName.getName(), 5000);
try { try {
// put one row // put one row

View File

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

View File

@ -174,7 +174,7 @@ public class TestReplicasClient {
TestRegionServerNoMaster.stopMasterAndAssignMeta(HTU); TestRegionServerNoMaster.stopMasterAndAssignMeta(HTU);
Configuration c = new Configuration(HTU.getConfiguration()); Configuration c = new Configuration(HTU.getConfiguration());
c.setInt(HConstants.HBASE_CLIENT_RETRIES_NUMBER, 1); c.setInt(HConstants.HBASE_CLIENT_RETRIES_NUMBER, 1);
HBaseAdmin ha = new HBaseAdmin(c); Admin ha = new HBaseAdmin(c);
for (boolean masterRuns = true; masterRuns; ) { for (boolean masterRuns = true; masterRuns; ) {
Thread.sleep(100); Thread.sleep(100);
try { 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 // change one of the connection properties so we get a new HConnection with our configuration
conf.setInt(HConstants.HBASE_RPC_TIMEOUT_KEY, HConstants.DEFAULT_HBASE_RPC_TIMEOUT + 1); conf.setInt(HConstants.HBASE_RPC_TIMEOUT_KEY, HConstants.DEFAULT_HBASE_RPC_TIMEOUT + 1);
HTable table = new HTable(conf, name); Table table = new HTable(conf, name);
table.setAutoFlushTo(false); table.setAutoFlushTo(false);
byte[] row = Bytes.toBytes("row"); byte[] row = Bytes.toBytes("row");
Put p = new Put(row); Put p = new Put(row);
@ -188,7 +188,7 @@ public class TestRpcControllerFactory {
table.close(); 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); ResultScanner results = table.getScanner(scan);
results.next(); results.next();
results.close(); results.close();

View File

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

View File

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

View File

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

View File

@ -169,7 +169,7 @@ public class TestSnapshotMetadata {
assertTrue(htd.getConfiguration().size() > 0); assertTrue(htd.getConfiguration().size() > 0);
admin.createTable(htd); admin.createTable(htd);
HTable original = new HTable(UTIL.getConfiguration(), originalTableName); Table original = new HTable(UTIL.getConfiguration(), originalTableName);
originalTableName = TableName.valueOf(sourceTableNameAsString); originalTableName = TableName.valueOf(sourceTableNameAsString);
originalTableDescriptor = admin.getTableDescriptor(originalTableName); originalTableDescriptor = admin.getTableDescriptor(originalTableName);
originalTableDescription = originalTableDescriptor.toStringCustomizedValues(); originalTableDescription = originalTableDescriptor.toStringCustomizedValues();
@ -201,7 +201,7 @@ public class TestSnapshotMetadata {
familiesList, snapshotNameAsString, rootDir, fs, /* onlineSnapshot= */ false); familiesList, snapshotNameAsString, rootDir, fs, /* onlineSnapshot= */ false);
admin.cloneSnapshot(snapshotName, clonedTableName); admin.cloneSnapshot(snapshotName, clonedTableName);
HTable clonedTable = new HTable(UTIL.getConfiguration(), clonedTableName); Table clonedTable = new HTable(UTIL.getConfiguration(), clonedTableName);
HTableDescriptor cloneHtd = admin.getTableDescriptor(clonedTableName); HTableDescriptor cloneHtd = admin.getTableDescriptor(clonedTableName);
assertEquals( assertEquals(
originalTableDescription.replace(originalTableName.getNameAsString(),clonedTableNameAsString), originalTableDescription.replace(originalTableName.getNameAsString(),clonedTableNameAsString),
@ -312,7 +312,7 @@ public class TestSnapshotMetadata {
admin.enableTable(originalTableName); admin.enableTable(originalTableName);
// verify that the descrption is reverted // verify that the descrption is reverted
HTable original = new HTable(UTIL.getConfiguration(), originalTableName); Table original = new HTable(UTIL.getConfiguration(), originalTableName);
try { try {
assertTrue(originalTableDescriptor.equals(admin.getTableDescriptor(originalTableName))); assertTrue(originalTableDescriptor.equals(admin.getTableDescriptor(originalTableName)));
assertTrue(originalTableDescriptor.equals(original.getTableDescriptor())); assertTrue(originalTableDescriptor.equals(original.getTableDescriptor()));

View File

@ -96,7 +96,7 @@ public class TestTimestampsFilter {
Cell kvs[]; Cell kvs[];
// create table; set versions to max... // 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 rowIdx = 0; rowIdx < 5; rowIdx++) {
for (int colIdx = 0; colIdx < 5; colIdx++) { for (int colIdx = 0; colIdx < 5; colIdx++) {
@ -171,7 +171,7 @@ public class TestTimestampsFilter {
byte [][] FAMILIES = new byte[][] { FAMILY }; byte [][] FAMILIES = new byte[][] { FAMILY };
// create table; set versions to max... // 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")); Put p = new Put(Bytes.toBytes("row"));
p.add(FAMILY, Bytes.toBytes("column0"), 3, Bytes.toBytes("value0-3")); p.add(FAMILY, Bytes.toBytes("column0"), 3, Bytes.toBytes("value0-3"));
@ -231,7 +231,7 @@ public class TestTimestampsFilter {
byte [][] FAMILIES = new byte[][] { FAMILY }; byte [][] FAMILIES = new byte[][] { FAMILY };
// create table; set versions to max... // 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. // For row:0, col:0: insert versions 1 through 5.
putNVersions(ht, FAMILY, 0, 0, 1, 5); putNVersions(ht, FAMILY, 0, 0, 1, 5);
@ -254,7 +254,7 @@ public class TestTimestampsFilter {
ht.close(); 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 rowIdx = 0; rowIdx < 5; rowIdx++) {
for (int colIdx = 0; colIdx < 5; colIdx++) { for (int colIdx = 0; colIdx < 5; colIdx++) {
// ask for versions that exist. // ask for versions that exist.
@ -313,7 +313,7 @@ public class TestTimestampsFilter {
* versions for the row/column specified by rowIdx & colIdx. * 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) int colIdx, List<Long> versions)
throws IOException { throws IOException {
byte row[] = Bytes.toBytes("row:" + rowIdx); 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 * Uses the TimestampFilter on a Scan to request a specified list of
* versions for the rows from startRowIdx to endRowIdx (both inclusive). * 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) int endRowIdx, List<Long> versions)
throws IOException { throws IOException {
byte startRow[] = Bytes.toBytes("row:" + startRowIdx); byte startRow[] = Bytes.toBytes("row:" + startRowIdx);
@ -349,7 +349,7 @@ public class TestTimestampsFilter {
* Insert in specific row/column versions with timestamps * Insert in specific row/column versions with timestamps
* versionStart..versionEnd. * 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) long versionStart, long versionEnd)
throws IOException { throws IOException {
byte row[] = Bytes.toBytes("row:" + rowIdx); byte row[] = Bytes.toBytes("row:" + rowIdx);
@ -368,7 +368,7 @@ public class TestTimestampsFilter {
* For row/column specified by rowIdx/colIdx, delete the cell * For row/column specified by rowIdx/colIdx, delete the cell
* corresponding to the specified version. * 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) int colIdx, long version)
throws IOException { throws IOException {
byte row[] = Bytes.toBytes("row:" + rowIdx); 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.HTable;
import org.apache.hadoop.hbase.client.Put; import org.apache.hadoop.hbase.client.Put;
import org.apache.hadoop.hbase.client.RetriesExhaustedWithDetailsException; 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.coprocessor.CoprocessorHost;
import org.apache.hadoop.hbase.util.Bytes; import org.apache.hadoop.hbase.util.Bytes;
import org.junit.After; import org.junit.After;
@ -80,7 +81,7 @@ public class TestConstraint {
Constraints.add(desc, CheckWasRunConstraint.class); Constraints.add(desc, CheckWasRunConstraint.class);
util.getHBaseAdmin().createTable(desc); util.getHBaseAdmin().createTable(desc);
HTable table = new HTable(util.getConfiguration(), tableName); Table table = new HTable(util.getConfiguration(), tableName);
try { try {
// test that we don't fail on a valid put // test that we don't fail on a valid put
Put put = new Put(row1); Put put = new Put(row1);
@ -112,7 +113,7 @@ public class TestConstraint {
Constraints.add(desc, AllFailConstraint.class); Constraints.add(desc, AllFailConstraint.class);
util.getHBaseAdmin().createTable(desc); 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 // test that we do fail on violation
Put put = new Put(row1); Put put = new Put(row1);
@ -155,7 +156,7 @@ public class TestConstraint {
Constraints.disableConstraint(desc, AllFailConstraint.class); Constraints.disableConstraint(desc, AllFailConstraint.class);
util.getHBaseAdmin().createTable(desc); util.getHBaseAdmin().createTable(desc);
HTable table = new HTable(util.getConfiguration(), tableName); Table table = new HTable(util.getConfiguration(), tableName);
try { try {
// test that we don't fail because its disabled // test that we don't fail because its disabled
Put put = new Put(row1); Put put = new Put(row1);
@ -187,7 +188,7 @@ public class TestConstraint {
Constraints.disable(desc); Constraints.disable(desc);
util.getHBaseAdmin().createTable(desc); util.getHBaseAdmin().createTable(desc);
HTable table = new HTable(util.getConfiguration(), tableName); Table table = new HTable(util.getConfiguration(), tableName);
try { try {
// test that we do fail on violation // test that we do fail on violation
Put put = new Put(row1); Put put = new Put(row1);
@ -219,7 +220,7 @@ public class TestConstraint {
CheckWasRunConstraint.wasRun = false; CheckWasRunConstraint.wasRun = false;
util.getHBaseAdmin().createTable(desc); 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 // test that we do fail on violation
Put put = new Put(row1); Put put = new Put(row1);

View File

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

View File

@ -31,6 +31,7 @@ import java.util.NavigableMap;
import java.util.TreeMap; import java.util.TreeMap;
import org.apache.hadoop.hbase.client.Admin; import org.apache.hadoop.hbase.client.Admin;
import org.apache.hadoop.hbase.client.Table;
import org.apache.hadoop.hbase.util.ByteStringer; import org.apache.hadoop.hbase.util.ByteStringer;
import org.apache.commons.logging.Log; import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory; import org.apache.commons.logging.LogFactory;
@ -92,14 +93,14 @@ public class TestCoprocessorEndpoint {
conf.setStrings(CoprocessorHost.MASTER_COPROCESSOR_CONF_KEY, conf.setStrings(CoprocessorHost.MASTER_COPROCESSOR_CONF_KEY,
ProtobufCoprocessorService.class.getName()); ProtobufCoprocessorService.class.getName());
util.startMiniCluster(2); util.startMiniCluster(2);
HBaseAdmin admin = new HBaseAdmin(conf); Admin admin = new HBaseAdmin(conf);
HTableDescriptor desc = new HTableDescriptor(TEST_TABLE); HTableDescriptor desc = new HTableDescriptor(TEST_TABLE);
desc.addFamily(new HColumnDescriptor(TEST_FAMILY)); desc.addFamily(new HColumnDescriptor(TEST_FAMILY));
admin.createTable(desc, new byte[][]{ROWS[rowSeperator1], ROWS[rowSeperator2]}); admin.createTable(desc, new byte[][]{ROWS[rowSeperator1], ROWS[rowSeperator2]});
util.waitUntilAllRegionsAssigned(TEST_TABLE); util.waitUntilAllRegionsAssigned(TEST_TABLE);
admin.close(); admin.close();
HTable table = new HTable(conf, TEST_TABLE); Table table = new HTable(conf, TEST_TABLE);
for (int i = 0; i < ROWSIZE; i++) { for (int i = 0; i < ROWSIZE; i++) {
Put put = new Put(ROWS[i]); Put put = new Put(ROWS[i]);
put.add(TEST_FAMILY, TEST_QUALIFIER, Bytes.toBytes(i)); put.add(TEST_FAMILY, TEST_QUALIFIER, Bytes.toBytes(i));
@ -113,7 +114,7 @@ public class TestCoprocessorEndpoint {
util.shutdownMiniCluster(); 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) final byte [] qualifier, final byte [] start, final byte [] end)
throws ServiceException, Throwable { throws ServiceException, Throwable {
return table.coprocessorService(ColumnAggregationProtos.ColumnAggregationService.class, return table.coprocessorService(ColumnAggregationProtos.ColumnAggregationService.class,
@ -138,7 +139,7 @@ public class TestCoprocessorEndpoint {
@Test @Test
public void testAggregation() throws Throwable { 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, Map<byte[], Long> results = sum(table, TEST_FAMILY, TEST_QUALIFIER,
ROWS[0], ROWS[ROWS.length-1]); ROWS[0], ROWS[ROWS.length-1]);
int sumResult = 0; int sumResult = 0;
@ -297,7 +298,7 @@ public class TestCoprocessorEndpoint {
Configuration configuration = new Configuration(util.getConfiguration()); Configuration configuration = new Configuration(util.getConfiguration());
// Make it not retry forever // Make it not retry forever
configuration.setInt(HConstants.HBASE_CLIENT_RETRIES_NUMBER, 1); configuration.setInt(HConstants.HBASE_CLIENT_RETRIES_NUMBER, 1);
HTable table = new HTable(configuration, TEST_TABLE); Table table = new HTable(configuration, TEST_TABLE);
try { try {
CoprocessorRpcChannel protocol = table.coprocessorService(ROWS[0]); 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.Delete;
import org.apache.hadoop.hbase.client.Durability; import org.apache.hadoop.hbase.client.Durability;
import org.apache.hadoop.hbase.client.Get; import org.apache.hadoop.hbase.client.Get;
import org.apache.hadoop.hbase.client.HTable;
import org.apache.hadoop.hbase.client.HTableInterface; import org.apache.hadoop.hbase.client.HTableInterface;
import org.apache.hadoop.hbase.client.Increment; import org.apache.hadoop.hbase.client.Increment;
import org.apache.hadoop.hbase.client.Put; 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.Row;
import org.apache.hadoop.hbase.client.RowMutations; import org.apache.hadoop.hbase.client.RowMutations;
import org.apache.hadoop.hbase.client.Scan; import org.apache.hadoop.hbase.client.Scan;
import org.apache.hadoop.hbase.client.Table;
import org.apache.hadoop.hbase.client.coprocessor.Batch; import org.apache.hadoop.hbase.client.coprocessor.Batch;
import org.apache.hadoop.hbase.ipc.CoprocessorRpcChannel; import org.apache.hadoop.hbase.ipc.CoprocessorRpcChannel;
import org.apache.hadoop.hbase.master.MasterCoprocessorHost; import org.apache.hadoop.hbase.master.MasterCoprocessorHost;
@ -86,7 +86,7 @@ public class TestHTableWrapper {
} }
private HTableInterface hTableInterface; private HTableInterface hTableInterface;
private HTable table; private Table table;
@BeforeClass @BeforeClass
public static void setupBeforeClass() throws Exception { 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.TableName;
import org.apache.hadoop.hbase.client.Admin; import org.apache.hadoop.hbase.client.Admin;
import org.apache.hadoop.hbase.client.Durability; import org.apache.hadoop.hbase.client.Durability;
import org.apache.hadoop.hbase.client.HBaseAdmin;
import org.apache.hadoop.hbase.client.HTable; import org.apache.hadoop.hbase.client.HTable;
import org.apache.hadoop.hbase.client.HTableInterface;
import org.apache.hadoop.hbase.client.Put; import org.apache.hadoop.hbase.client.Put;
import org.apache.hadoop.hbase.client.Result; import org.apache.hadoop.hbase.client.Result;
import org.apache.hadoop.hbase.client.ResultScanner; import org.apache.hadoop.hbase.client.ResultScanner;
import org.apache.hadoop.hbase.client.Scan; import org.apache.hadoop.hbase.client.Scan;
import org.apache.hadoop.hbase.client.Table;
import org.apache.hadoop.hbase.regionserver.wal.WALEdit; import org.apache.hadoop.hbase.regionserver.wal.WALEdit;
import org.apache.hadoop.hbase.util.Threads; import org.apache.hadoop.hbase.util.Threads;
import org.junit.After; import org.junit.After;
@ -69,7 +68,7 @@ public class TestOpenTableInCoprocessor {
@Override @Override
public void prePut(final ObserverContext<RegionCoprocessorEnvironment> e, final Put put, public void prePut(final ObserverContext<RegionCoprocessorEnvironment> e, final Put put,
final WALEdit edit, final Durability durability) throws IOException { final WALEdit edit, final Durability durability) throws IOException {
HTableInterface table = e.getEnvironment().getTable(otherTable); Table table = e.getEnvironment().getTable(otherTable);
table.put(put); table.put(put);
table.flushCommits(); table.flushCommits();
completed[0] = true; completed[0] = true;
@ -102,7 +101,7 @@ public class TestOpenTableInCoprocessor {
@Override @Override
public void prePut(final ObserverContext<RegionCoprocessorEnvironment> e, final Put put, public void prePut(final ObserverContext<RegionCoprocessorEnvironment> e, final Put put,
final WALEdit edit, final Durability durability) throws IOException { 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' }); Put p = new Put(new byte[] { 'a' });
p.add(family, null, new byte[] { 'a' }); p.add(family, null, new byte[] { 'a' });
try { try {
@ -162,14 +161,14 @@ public class TestOpenTableInCoprocessor {
admin.createTable(primary); admin.createTable(primary);
admin.createTable(other); admin.createTable(other);
HTable table = new HTable(UTIL.getConfiguration(), "primary"); Table table = new HTable(UTIL.getConfiguration(), "primary");
Put p = new Put(new byte[] { 'a' }); Put p = new Put(new byte[] { 'a' });
p.add(family, null, new byte[] { 'a' }); p.add(family, null, new byte[] { 'a' });
table.put(p); table.put(p);
table.flushCommits(); table.flushCommits();
table.close(); 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]); assertTrue("Didn't complete update to target table!", completeCheck[0]);
assertEquals("Didn't find inserted row", 1, getKeyValueCount(target)); assertEquals("Didn't find inserted row", 1, getKeyValueCount(target));
target.close(); target.close();
@ -181,7 +180,7 @@ public class TestOpenTableInCoprocessor {
* @return number of keyvalues over all rows in the table * @return number of keyvalues over all rows in the table
* @throws IOException * @throws IOException
*/ */
private int getKeyValueCount(HTable table) throws IOException { private int getKeyValueCount(Table table) throws IOException {
Scan scan = new Scan(); Scan scan = new Scan();
scan.setMaxVersions(Integer.MAX_VALUE - 1); 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.Cell;
import org.apache.hadoop.hbase.HBaseConfiguration; import org.apache.hadoop.hbase.HBaseConfiguration;
import org.apache.hadoop.hbase.HBaseTestingUtility; import org.apache.hadoop.hbase.HBaseTestingUtility;
import org.apache.hadoop.hbase.KeyValue;
import org.apache.hadoop.hbase.MediumTests; import org.apache.hadoop.hbase.MediumTests;
import org.apache.hadoop.hbase.TableName; import org.apache.hadoop.hbase.TableName;
import org.apache.hadoop.hbase.client.Admin; import org.apache.hadoop.hbase.client.Admin;
import org.apache.hadoop.hbase.client.Delete; import org.apache.hadoop.hbase.client.Delete;
import org.apache.hadoop.hbase.client.Get; import org.apache.hadoop.hbase.client.Get;
import org.apache.hadoop.hbase.client.HBaseAdmin;
import org.apache.hadoop.hbase.client.HTable; import org.apache.hadoop.hbase.client.HTable;
import org.apache.hadoop.hbase.client.Put; import org.apache.hadoop.hbase.client.Put;
import org.apache.hadoop.hbase.client.Result; import org.apache.hadoop.hbase.client.Result;
import org.apache.hadoop.hbase.client.Durability; 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.regionserver.wal.WALEdit;
import org.apache.hadoop.hbase.util.Bytes; import org.apache.hadoop.hbase.util.Bytes;
import org.apache.hadoop.hbase.util.EnvironmentEdgeManager; import org.apache.hadoop.hbase.util.EnvironmentEdgeManager;
@ -93,7 +92,7 @@ public class TestRegionObserverBypass {
*/ */
@Test @Test
public void testSimple() throws Exception { public void testSimple() throws Exception {
HTable t = new HTable(util.getConfiguration(), tableName); Table t = new HTable(util.getConfiguration(), tableName);
Put p = new Put(row1); Put p = new Put(row1);
p.add(test,dummy,dummy); p.add(test,dummy,dummy);
// before HBASE-4331, this would throw an exception // before HBASE-4331, this would throw an exception
@ -112,7 +111,7 @@ public class TestRegionObserverBypass {
//previous deletes will eclipse successive puts having the same timestamp //previous deletes will eclipse successive puts having the same timestamp
EnvironmentEdgeManagerTestHelper.injectEdge(new IncrementingEnvironmentEdge()); EnvironmentEdgeManagerTestHelper.injectEdge(new IncrementingEnvironmentEdge());
HTable t = new HTable(util.getConfiguration(), tableName); Table t = new HTable(util.getConfiguration(), tableName);
List<Put> puts = new ArrayList<Put>(); List<Put> puts = new ArrayList<Put>();
Put p = new Put(row1); Put p = new Put(row1);
p.add(dummy,dummy,dummy); p.add(dummy,dummy,dummy);
@ -197,7 +196,7 @@ public class TestRegionObserverBypass {
EnvironmentEdgeManager.reset(); 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); Get g = new Get(row);
Result r = t.get(g); Result r = t.get(g);
assertEquals(count, r.size()); 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.ResultScanner;
import org.apache.hadoop.hbase.client.RowMutations; import org.apache.hadoop.hbase.client.RowMutations;
import org.apache.hadoop.hbase.client.Scan; import org.apache.hadoop.hbase.client.Scan;
import org.apache.hadoop.hbase.client.Table;
import org.apache.hadoop.hbase.io.hfile.CacheConfig; import org.apache.hadoop.hbase.io.hfile.CacheConfig;
import org.apache.hadoop.hbase.io.hfile.HFile; import org.apache.hadoop.hbase.io.hfile.HFile;
import org.apache.hadoop.hbase.io.hfile.HFileContext; import org.apache.hadoop.hbase.io.hfile.HFileContext;
@ -114,7 +115,7 @@ public class TestRegionObserverInterface {
TableName tableName = TableName.valueOf(TEST_TABLE.getNameAsString() + ".testRegionObserver"); TableName tableName = TableName.valueOf(TEST_TABLE.getNameAsString() + ".testRegionObserver");
// recreate table every time in order to reset the status of the // recreate table every time in order to reset the status of the
// coprocessor. // coprocessor.
HTable table = util.createTable(tableName, new byte[][] {A, B, C}); Table table = util.createTable(tableName, new byte[][] {A, B, C});
try { try {
verifyMethodResult(SimpleRegionObserver.class, new String[] { "hadPreGet", "hadPostGet", verifyMethodResult(SimpleRegionObserver.class, new String[] { "hadPreGet", "hadPostGet",
"hadPrePut", "hadPostPut", "hadDelete", "hadPostStartRegionOperation", "hadPrePut", "hadPostPut", "hadDelete", "hadPostStartRegionOperation",
@ -176,7 +177,7 @@ public class TestRegionObserverInterface {
@Test @Test
public void testRowMutation() throws IOException { public void testRowMutation() throws IOException {
TableName tableName = TableName.valueOf(TEST_TABLE.getNameAsString() + ".testRowMutation"); 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 { try {
verifyMethodResult(SimpleRegionObserver.class, verifyMethodResult(SimpleRegionObserver.class,
new String[] {"hadPreGet", "hadPostGet", "hadPrePut", "hadPostPut", new String[] {"hadPreGet", "hadPostGet", "hadPrePut", "hadPostPut",
@ -213,7 +214,7 @@ public class TestRegionObserverInterface {
@Test @Test
public void testIncrementHook() throws IOException { public void testIncrementHook() throws IOException {
TableName tableName = TableName.valueOf(TEST_TABLE.getNameAsString() + ".testIncrementHook"); 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 { try {
Increment inc = new Increment(Bytes.toBytes(0)); Increment inc = new Increment(Bytes.toBytes(0));
inc.addColumn(A, A, 1); inc.addColumn(A, A, 1);
@ -241,7 +242,7 @@ public class TestRegionObserverInterface {
public void testCheckAndPutHooks() throws IOException { public void testCheckAndPutHooks() throws IOException {
TableName tableName = TableName tableName =
TableName.valueOf(TEST_TABLE.getNameAsString() + ".testCheckAndPutHooks"); 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 { try {
Put p = new Put(Bytes.toBytes(0)); Put p = new Put(Bytes.toBytes(0));
p.add(A, A, A); p.add(A, A, A);
@ -272,7 +273,7 @@ public class TestRegionObserverInterface {
public void testCheckAndDeleteHooks() throws IOException { public void testCheckAndDeleteHooks() throws IOException {
TableName tableName = TableName tableName =
TableName.valueOf(TEST_TABLE.getNameAsString() + ".testCheckAndDeleteHooks"); 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 { try {
Put p = new Put(Bytes.toBytes(0)); Put p = new Put(Bytes.toBytes(0));
p.add(A, A, A); p.add(A, A, A);
@ -302,7 +303,7 @@ public class TestRegionObserverInterface {
@Test @Test
public void testAppendHook() throws IOException { public void testAppendHook() throws IOException {
TableName tableName = TableName.valueOf(TEST_TABLE.getNameAsString() + ".testAppendHook"); 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 { try {
Append app = new Append(Bytes.toBytes(0)); Append app = new Append(Bytes.toBytes(0));
app.add(A, A, A); app.add(A, A, A);
@ -341,7 +342,7 @@ public class TestRegionObserverInterface {
new Boolean[] {false, false, false, false} 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 put = new Put(ROW);
put.add(A, A, A); put.add(A, A, A);
table.put(put); table.put(put);
@ -391,7 +392,7 @@ public class TestRegionObserverInterface {
new Boolean[] {false, false} new Boolean[] {false, false}
); );
HTable table = new HTable(util.getConfiguration(), tableName); Table table = new HTable(util.getConfiguration(), tableName);
Put put = new Put(ROW); Put put = new Put(ROW);
put.add(A, A, A); put.add(A, A, A);
table.put(put); table.put(put);
@ -498,7 +499,7 @@ public class TestRegionObserverInterface {
htd.addCoprocessor(EvenOnlyCompactor.class.getName()); htd.addCoprocessor(EvenOnlyCompactor.class.getName());
admin.createTable(htd); admin.createTable(htd);
HTable table = new HTable(util.getConfiguration(), compactTable); Table table = new HTable(util.getConfiguration(), compactTable);
for (long i=1; i<=10; i++) { for (long i=1; i<=10; i++) {
byte[] iBytes = Bytes.toBytes(i); byte[] iBytes = Bytes.toBytes(i);
Put put = new Put(iBytes); Put put = new Put(iBytes);
@ -560,7 +561,7 @@ public class TestRegionObserverInterface {
String testName = TestRegionObserverInterface.class.getName()+".bulkLoadHFileTest"; String testName = TestRegionObserverInterface.class.getName()+".bulkLoadHFileTest";
TableName tableName = TableName.valueOf(TEST_TABLE.getNameAsString() + ".bulkLoadHFileTest"); TableName tableName = TableName.valueOf(TEST_TABLE.getNameAsString() + ".bulkLoadHFileTest");
Configuration conf = util.getConfiguration(); 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 { try {
verifyMethodResult(SimpleRegionObserver.class, verifyMethodResult(SimpleRegionObserver.class,
new String[] {"hadPreBulkLoadHFile", "hadPostBulkLoadHFile"}, 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.Put;
import org.apache.hadoop.hbase.client.Result; import org.apache.hadoop.hbase.client.Result;
import org.apache.hadoop.hbase.client.Scan; import org.apache.hadoop.hbase.client.Scan;
import org.apache.hadoop.hbase.client.Table;
import org.apache.hadoop.hbase.filter.FilterBase; import org.apache.hadoop.hbase.filter.FilterBase;
import org.apache.hadoop.hbase.regionserver.HRegion; import org.apache.hadoop.hbase.regionserver.HRegion;
import org.apache.hadoop.hbase.regionserver.HRegionServer; import org.apache.hadoop.hbase.regionserver.HRegionServer;
@ -257,7 +258,7 @@ public class TestRegionObserverScannerOpenHook {
Admin admin = UTIL.getHBaseAdmin(); Admin admin = UTIL.getHBaseAdmin();
admin.createTable(desc); 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 a row and flush it to disk
Put put = new Put(ROW); 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.MiniHBaseCluster;
import org.apache.hadoop.hbase.TableName; import org.apache.hadoop.hbase.TableName;
import org.apache.hadoop.hbase.MetaTableAccessor; 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.HBaseAdmin;
import org.apache.hadoop.hbase.client.Mutation; import org.apache.hadoop.hbase.client.Mutation;
import org.apache.hadoop.hbase.regionserver.HRegion; import org.apache.hadoop.hbase.regionserver.HRegion;
@ -74,7 +75,7 @@ public class TestRegionServerObserver {
// Start the cluster // Start the cluster
HBaseTestingUtility TEST_UTIL = new HBaseTestingUtility(conf); HBaseTestingUtility TEST_UTIL = new HBaseTestingUtility(conf);
TEST_UTIL.startMiniCluster(NUM_MASTERS, NUM_RS); TEST_UTIL.startMiniCluster(NUM_MASTERS, NUM_RS);
HBaseAdmin admin = new HBaseAdmin(conf); Admin admin = new HBaseAdmin(conf);
try { try {
MiniHBaseCluster cluster = TEST_UTIL.getHBaseCluster(); MiniHBaseCluster cluster = TEST_UTIL.getHBaseCluster();
HRegionServer regionServer = cluster.getRegionServer(0); HRegionServer regionServer = cluster.getRegionServer(0);

View File

@ -34,6 +34,7 @@ import java.util.concurrent.atomic.AtomicInteger;
import java.util.concurrent.atomic.AtomicLong; import java.util.concurrent.atomic.AtomicLong;
import org.apache.hadoop.hbase.TableName; import org.apache.hadoop.hbase.TableName;
import org.apache.hadoop.hbase.client.Table;
import org.apache.hadoop.hbase.util.ByteStringer; import org.apache.hadoop.hbase.util.ByteStringer;
import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.hbase.Cell; 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.MediumTests;
import org.apache.hadoop.hbase.client.Delete; import org.apache.hadoop.hbase.client.Delete;
import org.apache.hadoop.hbase.client.Get; 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.IsolationLevel;
import org.apache.hadoop.hbase.client.Mutation; import org.apache.hadoop.hbase.client.Mutation;
import org.apache.hadoop.hbase.client.Put; import org.apache.hadoop.hbase.client.Put;
@ -107,7 +107,7 @@ public class TestRowProcessorEndpoint {
private static volatile int expectedCounter = 0; private static volatile int expectedCounter = 0;
private static int rowSize, row2Size; 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 static boolean swapped = false;
private volatile CountDownLatch startSignal; private volatile CountDownLatch startSignal;
private volatile CountDownLatch doneSignal; 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); CoprocessorRpcChannel channel = table.coprocessorService(ROW);
RowProcessorEndpoint.IncrementCounterProcessor processor = RowProcessorEndpoint.IncrementCounterProcessor processor =
new RowProcessorEndpoint.IncrementCounterProcessor(ROW); 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); CoprocessorRpcChannel channel = table.coprocessorService(ROW);
RowProcessorEndpoint.RowSwapProcessor processor = RowProcessorEndpoint.RowSwapProcessor processor =
new RowProcessorEndpoint.RowSwapProcessor(ROW, ROW2); new RowProcessorEndpoint.RowSwapProcessor(ROW, ROW2);

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