From b175042917f88aebcffa2b554da73273620af5b1 Mon Sep 17 00:00:00 2001 From: Enis Soztutar Date: Mon, 8 Sep 2014 23:39:34 -0700 Subject: [PATCH] HBASE-11679 Replace HTable with HTableInterface where backwards-compatible (Carter) --- .../hadoop/hbase/MetaTableAccessor.java | 29 +-- .../apache/hadoop/hbase/client/HTable.java | 3 +- .../hadoop/hbase/client/HTableUtil.java | 5 +- .../client/coprocessor/AggregationClient.java | 35 +-- .../coprocessor/SecureBulkLoadClient.java | 6 +- .../security/access/AccessControlClient.java | 10 +- .../security/visibility/VisibilityClient.java | 7 +- .../hbase/client/TestClientNoCluster.java | 10 +- .../hbase/client/TestSnapshotFromAdmin.java | 6 +- .../example/TestBulkDeleteProtocol.java | 23 +- .../example/TestRowCountEndpoint.java | 7 +- .../TestZooKeeperScanPolicyObserver.java | 3 +- .../hadoop/hbase/DistributedHBaseCluster.java | 3 +- .../hbase/IntegrationTestLazyCfLoading.java | 3 +- .../mapreduce/IntegrationTestImportTsv.java | 3 +- .../hbase/mttr/IntegrationTestMTTR.java | 8 +- .../test/IntegrationTestBigLinkedList.java | 14 +- ...rationTestBigLinkedListWithVisibility.java | 10 +- .../test/IntegrationTestLoadAndVerify.java | 2 +- ...TimeBoundedRequestsWithRegionReplicas.java | 4 +- ...onTestWithCellVisibilityLoadAndVerify.java | 3 +- .../IntegrationTestSendTraceRequests.java | 5 +- .../hadoop/hbase/LocalHBaseCluster.java | 3 +- .../hbase/coprocessor/CoprocessorHost.java | 1 + .../hbase/mapred/HRegionPartitioner.java | 3 +- .../hbase/mapred/TableOutputFormat.java | 5 +- .../hbase/mapred/TableRecordReader.java | 3 +- .../hbase/mapred/TableRecordReaderImpl.java | 5 +- .../DefaultVisibilityExpressionResolver.java | 3 +- .../hbase/mapreduce/HFileOutputFormat.java | 9 +- .../hbase/mapreduce/HFileOutputFormat2.java | 12 +- .../hbase/mapreduce/HRegionPartitioner.java | 3 +- .../hadoop/hbase/mapreduce/ImportTsv.java | 3 +- .../mapreduce/LoadIncrementalHFiles.java | 11 +- .../mapreduce/MultiTableInputFormatBase.java | 3 +- .../hbase/mapreduce/TableInputFormatBase.java | 2 - .../hbase/mapreduce/TableOutputFormat.java | 5 +- .../hbase/mapreduce/TableRecordReader.java | 3 +- .../mapreduce/TableRecordReaderImpl.java | 5 +- .../replication/VerifyReplication.java | 3 +- .../hbase/master/HMasterCommandLine.java | 4 +- .../master/RegionPlacementMaintainer.java | 5 +- .../hbase/master/TableNamespaceManager.java | 11 +- .../balancer/FavoredNodeAssignmentHelper.java | 3 +- .../master/handler/ModifyTableHandler.java | 3 +- .../regionserver/ReplicationSink.java | 4 +- .../apache/hadoop/hbase/rest/RESTServlet.java | 4 +- .../apache/hadoop/hbase/rest/RowResource.java | 12 +- .../hadoop/hbase/rest/RowResultGenerator.java | 4 +- .../hbase/rest/ScannerResultGenerator.java | 4 +- .../hadoop/hbase/rest/SchemaResource.java | 4 +- .../hadoop/hbase/rest/TableResource.java | 4 +- .../security/access/AccessControlLists.java | 15 +- .../hbase/security/token/TokenUtil.java | 3 +- .../org/apache/hadoop/hbase/tool/Canary.java | 9 +- .../apache/hadoop/hbase/util/HBaseFsck.java | 8 +- .../hadoop/hbase/util/HBaseFsckRepair.java | 13 +- .../org/apache/hadoop/hbase/util/HMerge.java | 6 +- .../hbase/util/RegionSizeCalculator.java | 4 +- .../apache/hadoop/hbase/HBaseTestCase.java | 5 +- .../hadoop/hbase/HBaseTestingUtility.java | 32 +-- .../hadoop/hbase/PerformanceEvaluation.java | 6 +- .../hbase/ScanPerformanceEvaluation.java | 3 +- .../hadoop/hbase/TestAcidGuarantees.java | 7 +- .../hadoop/hbase/TestHBaseTestingUtility.java | 6 +- .../hadoop/hbase/TestMetaTableAccessor.java | 5 +- .../hadoop/hbase/TestMultiVersions.java | 8 +- .../apache/hadoop/hbase/TestNamespace.java | 3 +- .../hadoop/hbase/TestRegionRebalancing.java | 6 +- .../apache/hadoop/hbase/TestZooKeeper.java | 16 +- .../apache/hadoop/hbase/client/TestAdmin.java | 20 +- .../client/TestClientOperationInterrupt.java | 6 +- .../client/TestClientScannerRPCTimeout.java | 4 +- .../hbase/client/TestClientTimeouts.java | 2 +- .../hbase/client/TestFromClientSide.java | 180 ++++++++-------- .../hbase/client/TestFromClientSide3.java | 10 +- .../hbase/client/TestHBaseAdminNoCluster.java | 3 +- .../apache/hadoop/hbase/client/TestHCM.java | 12 +- .../hadoop/hbase/client/TestHTablePool.java | 56 ++--- .../hbase/client/TestMultiParallel.java | 22 +- .../hbase/client/TestMultipleTimestamps.java | 32 +-- .../hbase/client/TestPutWithDelete.java | 2 +- .../hbase/client/TestReplicaWithCluster.java | 12 +- .../hbase/client/TestReplicasClient.java | 2 +- .../client/TestRpcControllerFactory.java | 4 +- .../hbase/client/TestScannerTimeout.java | 10 +- .../client/TestScannersFromClientSide.java | 8 +- .../client/TestSnapshotCloneIndependence.java | 2 +- .../hbase/client/TestSnapshotMetadata.java | 6 +- .../hbase/client/TestTimestampsFilter.java | 16 +- .../hbase/constraint/TestConstraint.java | 11 +- .../TestBatchCoprocessorEndpoint.java | 16 +- .../coprocessor/TestCoprocessorEndpoint.java | 11 +- .../hbase/coprocessor/TestHTableWrapper.java | 4 +- .../TestOpenTableInCoprocessor.java | 13 +- .../coprocessor/TestRegionObserverBypass.java | 9 +- .../TestRegionObserverInterface.java | 21 +- .../TestRegionObserverScannerOpenHook.java | 3 +- .../coprocessor/TestRegionServerObserver.java | 3 +- .../coprocessor/TestRowProcessorEndpoint.java | 8 +- .../hbase/filter/TestColumnRangeFilter.java | 7 +- .../filter/TestFilterWithScanLimits.java | 5 +- .../hbase/filter/TestFilterWrapper.java | 6 +- .../TestFuzzyRowAndColumnRangeFilter.java | 8 +- .../hadoop/hbase/fs/TestBlockReorder.java | 4 +- .../io/encoding/TestChangingEncoding.java | 5 +- .../hbase/io/encoding/TestPrefixTree.java | 3 +- .../hbase/mapred/TestTableInputFormat.java | 38 ++-- .../hbase/mapred/TestTableMapReduceUtil.java | 10 +- .../hbase/mapreduce/TestCellCounter.java | 4 +- .../hadoop/hbase/mapreduce/TestCopyTable.java | 14 +- .../mapreduce/TestHFileOutputFormat.java | 28 +-- .../mapreduce/TestHFileOutputFormat2.java | 28 +-- .../hbase/mapreduce/TestImportExport.java | 19 +- .../TestImportTSVWithOperationAttributes.java | 6 +- .../TestImportTSVWithVisibilityLabels.java | 8 +- .../hadoop/hbase/mapreduce/TestImportTsv.java | 3 +- .../mapreduce/TestLoadIncrementalHFiles.java | 4 +- ...estLoadIncrementalHFilesSplitRecovery.java | 5 +- .../TestMultithreadedTableMapper.java | 5 +- .../hbase/mapreduce/TestRowCounter.java | 6 +- .../mapreduce/TestTableMapReduceBase.java | 5 +- .../hbase/mapreduce/TestTimeRangeMapRed.java | 12 +- .../hadoop/hbase/mapreduce/TestWALPlayer.java | 6 +- .../hbase/master/TestAssignmentListener.java | 3 +- .../TestAssignmentManagerOnCluster.java | 27 +-- .../master/TestDistributedLogSplitting.java | 25 +-- .../hbase/master/TestMasterFailover.java | 1 + ...TestMasterOperationsForRegionReplicas.java | 6 +- .../TestMasterRestartAfterDisablingTable.java | 3 +- .../hbase/master/TestMasterTransitions.java | 5 +- .../hbase/master/TestRegionPlacement.java | 4 +- .../regionserver/TestCompactionState.java | 8 +- .../TestEncryptionKeyRotation.java | 3 +- .../TestEncryptionRandomKeying.java | 3 +- .../TestEndToEndSplitTransaction.java | 16 +- .../regionserver/TestFSErrorsExposed.java | 3 +- .../hbase/regionserver/TestHRegion.java | 4 +- .../regionserver/TestHRegionOnCluster.java | 4 +- .../regionserver/TestJoinedScanners.java | 5 +- .../TestRegionMergeTransactionOnCluster.java | 17 +- .../regionserver/TestRegionReplicas.java | 3 +- .../regionserver/TestRegionServerMetrics.java | 10 +- .../regionserver/TestSCVFWithMiniCluster.java | 4 +- .../regionserver/TestScannerWithBulkload.java | 3 +- .../TestServerCustomProtocol.java | 22 +- .../TestSplitTransactionOnCluster.java | 20 +- .../hadoop/hbase/regionserver/TestTags.java | 11 +- .../regionserver/wal/TestHLogFiltering.java | 4 +- .../regionserver/wal/TestLogRollAbort.java | 9 +- .../regionserver/wal/TestLogRollPeriod.java | 10 +- .../regionserver/wal/TestLogRolling.java | 23 +- .../hbase/regionserver/wal/TestWALReplay.java | 3 +- .../replication/TestMasterReplication.java | 26 +-- .../TestMultiSlaveReplication.java | 21 +- .../TestPerTableCFReplication.java | 31 +-- .../replication/TestReplicationBase.java | 8 +- .../TestReplicationSyncUpTool.java | 6 +- .../replication/TestReplicationWithTags.java | 8 +- .../regionserver/TestReplicationSink.java | 6 +- .../hadoop/hbase/rest/TestGzipFilter.java | 5 +- .../hbase/rest/TestScannerResource.java | 4 +- .../hbase/rest/TestScannersWithFilters.java | 4 +- .../hbase/rest/TestScannersWithLabels.java | 4 +- .../hbase/rest/client/TestRemoteTable.java | 6 +- .../hbase/security/access/SecureTestUtil.java | 15 +- .../access/TestAccessControlFilter.java | 11 +- .../security/access/TestAccessController.java | 65 +++--- .../access/TestAccessController2.java | 3 +- .../TestCellACLWithMultipleVersions.java | 60 +++--- .../hbase/security/access/TestCellACLs.java | 34 +-- .../access/TestNamespaceCommands.java | 7 +- .../access/TestScanEarlyTermination.java | 12 +- .../security/access/TestTablePermissions.java | 3 +- ...ExpAsStringVisibilityLabelServiceImpl.java | 3 +- .../TestEnforcingScanLabelGenerator.java | 5 +- .../visibility/TestVisibilityLabels.java | 39 ++-- .../TestVisibilityLabelsWithACL.java | 9 +- ...ilityLabelsWithDefaultVisLabelService.java | 4 +- .../TestVisibilityLabelsWithDeletes.java | 199 +++++++++--------- .../TestVisibilityLabelsWithSLGStack.java | 4 +- .../TestVisibilityWithCheckAuths.java | 13 +- .../hbase/snapshot/SnapshotTestingUtils.java | 6 +- .../hbase/snapshot/TestExportSnapshot.java | 10 +- .../hadoop/hbase/trace/TestHTraceHooks.java | 4 +- .../hadoop/hbase/util/LoadTestTool.java | 3 +- .../hbase/util/MultiThreadedReader.java | 9 +- .../util/MultiThreadedReaderWithACL.java | 7 +- .../hbase/util/MultiThreadedUpdater.java | 11 +- .../util/MultiThreadedUpdaterWithACL.java | 13 +- .../hbase/util/MultiThreadedWriter.java | 5 +- .../hbase/util/MultiThreadedWriterBase.java | 4 +- .../util/MultiThreadedWriterWithACL.java | 7 +- .../hadoop/hbase/util/RestartMetaTest.java | 3 +- .../hbase/util/TestCoprocessorScanPolicy.java | 5 +- .../hadoop/hbase/util/TestHBaseFsck.java | 29 +-- .../hbase/util/TestHBaseFsckEncryption.java | 3 +- .../hadoop/hbase/util/TestMergeTable.java | 3 +- .../util/TestMiniClusterLoadSequential.java | 5 +- .../hbase/util/TestProcessBasedCluster.java | 3 +- .../hbase/util/TestRegionSizeCalculator.java | 11 +- .../util/hbck/OfflineMetaRebuildTestCore.java | 13 +- .../hbase/thrift/IncrementCoalescer.java | 4 +- .../thrift2/ThriftHBaseServiceHandler.java | 33 +-- .../TestThriftHBaseServiceHandler.java | 3 +- ...stThriftHBaseServiceHandlerWithLabels.java | 3 +- 206 files changed, 1226 insertions(+), 1090 deletions(-) diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/MetaTableAccessor.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/MetaTableAccessor.java index c63e4c62d70..af9b5877644 100644 --- a/hbase-client/src/main/java/org/apache/hadoop/hbase/MetaTableAccessor.java +++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/MetaTableAccessor.java @@ -34,6 +34,7 @@ import org.apache.hadoop.hbase.client.RegionReplicaUtil; import org.apache.hadoop.hbase.client.Result; import org.apache.hadoop.hbase.client.ResultScanner; import org.apache.hadoop.hbase.client.Scan; +import org.apache.hadoop.hbase.client.Table; import org.apache.hadoop.hbase.ipc.CoprocessorRpcChannel; import org.apache.hadoop.hbase.protobuf.ProtobufUtil; import org.apache.hadoop.hbase.protobuf.generated.ClientProtos; @@ -174,7 +175,7 @@ public class MetaTableAccessor { * @throws IOException * @SuppressWarnings("deprecation") */ - private static HTable getHTable(final HConnection hConnection, + private static Table getHTable(final HConnection hConnection, final TableName tableName) throws IOException { // We used to pass whole CatalogTracker in here, now we just pass in HConnection @@ -190,7 +191,7 @@ public class MetaTableAccessor { * @return An {@link HTable} for hbase:meta * @throws IOException */ - static HTable getMetaHTable(final HConnection hConnection) + static Table getMetaHTable(final HConnection hConnection) throws IOException { return getHTable(hConnection, TableName.META_TABLE_NAME); } @@ -200,7 +201,7 @@ public class MetaTableAccessor { * @param g Get to run * @throws IOException */ - private static Result get(final HTable t, final Get g) throws IOException { + private static Result get(final Table t, final Get g) throws IOException { try { return t.get(g); } finally { @@ -599,7 +600,7 @@ public class MetaTableAccessor { scan.setCaching(caching); } scan.addFamily(HConstants.CATALOG_FAMILY); - HTable metaTable = getMetaHTable(hConnection); + Table metaTable = getMetaHTable(hConnection); ResultScanner scanner = null; try { scanner = metaTable.getScanner(scan); @@ -948,7 +949,7 @@ public class MetaTableAccessor { * @param p put to make * @throws IOException */ - private static void put(final HTable t, final Put p) throws IOException { + private static void put(final Table t, final Put p) throws IOException { try { t.put(p); } finally { @@ -964,7 +965,7 @@ public class MetaTableAccessor { */ public static void putsToMetaTable(final HConnection hConnection, final List ps) throws IOException { - HTable t = getMetaHTable(hConnection); + Table t = getMetaHTable(hConnection); try { t.put(ps); } finally { @@ -993,7 +994,7 @@ public class MetaTableAccessor { */ public static void deleteFromMetaTable(final HConnection hConnection, final List deletes) throws IOException { - HTable t = getMetaHTable(hConnection); + Table t = getMetaHTable(hConnection); try { t.delete(deletes); } finally { @@ -1036,7 +1037,7 @@ public class MetaTableAccessor { public static void mutateMetaTable(final HConnection hConnection, final List mutations) throws IOException { - HTable t = getMetaHTable(hConnection); + Table t = getMetaHTable(hConnection); try { t.batch(mutations); } catch (InterruptedException e) { @@ -1068,7 +1069,7 @@ public class MetaTableAccessor { * @param regionInfo region information * @throws IOException if problem connecting or updating meta */ - public static void addRegionToMeta(HTable meta, HRegionInfo regionInfo) throws IOException { + public static void addRegionToMeta(Table meta, HRegionInfo regionInfo) throws IOException { addRegionToMeta(meta, regionInfo, null, null); } @@ -1085,7 +1086,7 @@ public class MetaTableAccessor { * @param splitB second split daughter of the parent regionInfo * @throws IOException if problem connecting or updating meta */ - public static void addRegionToMeta(HTable meta, HRegionInfo regionInfo, + public static void addRegionToMeta(Table meta, HRegionInfo regionInfo, HRegionInfo splitA, HRegionInfo splitB) throws IOException { Put put = makePutFromRegionInfo(regionInfo); addDaughtersToPut(put, splitA, splitB); @@ -1109,7 +1110,7 @@ public class MetaTableAccessor { */ public static void addRegionToMeta(HConnection hConnection, HRegionInfo regionInfo, HRegionInfo splitA, HRegionInfo splitB) throws IOException { - HTable meta = getMetaHTable(hConnection); + Table meta = getMetaHTable(hConnection); try { addRegionToMeta(meta, regionInfo, splitA, splitB); } finally { @@ -1168,7 +1169,7 @@ public class MetaTableAccessor { */ public static void mergeRegions(final HConnection hConnection, HRegionInfo mergedRegion, HRegionInfo regionA, HRegionInfo regionB, ServerName sn) throws IOException { - HTable meta = getMetaHTable(hConnection); + Table meta = getMetaHTable(hConnection); try { HRegionInfo copyOfMerged = new HRegionInfo(mergedRegion); @@ -1208,7 +1209,7 @@ public class MetaTableAccessor { public static void splitRegion(final HConnection hConnection, HRegionInfo parent, HRegionInfo splitA, HRegionInfo splitB, ServerName sn) throws IOException { - HTable meta = getMetaHTable(hConnection); + Table meta = getMetaHTable(hConnection); try { HRegionInfo copyOfParent = new HRegionInfo(parent); copyOfParent.setOffline(true); @@ -1235,7 +1236,7 @@ public class MetaTableAccessor { /** * Performs an atomic multi-Mutate operation against the given table. */ - private static void multiMutate(HTable table, byte[] row, Mutation... mutations) + private static void multiMutate(Table table, byte[] row, Mutation... mutations) throws IOException { CoprocessorRpcChannel channel = table.coprocessorService(row); MultiRowMutationProtos.MutateRowsRequest.Builder mmrBuilder diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/HTable.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/HTable.java index 8a68c56ea68..40d46c4074b 100644 --- a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/HTable.java +++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/HTable.java @@ -22,7 +22,6 @@ import java.io.Closeable; import java.io.IOException; import java.io.InterruptedIOException; import java.util.ArrayList; -import java.util.Arrays; import java.util.Collections; import java.util.LinkedList; import java.util.List; @@ -1779,7 +1778,7 @@ public class HTable implements HTableInterface, RegionLocator { * @throws IOException */ public static void main(String[] args) throws IOException { - HTable t = new HTable(HBaseConfiguration.create(), args[0]); + Table t = new HTable(HBaseConfiguration.create(), args[0]); try { System.out.println(t.get(new Get(Bytes.toBytes(args[1])))); } finally { diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/HTableUtil.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/HTableUtil.java index 27aec10b3b3..04b3feaa252 100644 --- a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/HTableUtil.java +++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/HTableUtil.java @@ -19,7 +19,6 @@ package org.apache.hadoop.hbase.client; import org.apache.hadoop.classification.InterfaceAudience; -import org.apache.hadoop.classification.InterfaceStability; import org.apache.hadoop.hbase.HRegionLocation; import java.io.IOException; @@ -102,7 +101,7 @@ public class HTableUtil { } - private static Map> createRsPutMap(HTable htable, List puts) throws IOException { + private static Map> createRsPutMap(RegionLocator htable, List puts) throws IOException { Map> putMap = new HashMap>(); for (Put put: puts) { @@ -118,7 +117,7 @@ public class HTableUtil { return putMap; } - private static Map> createRsRowMap(HTable htable, List rows) throws IOException { + private static Map> createRsRowMap(RegionLocator htable, List rows) throws IOException { Map> rowMap = new HashMap>(); for (Row row: rows) { diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/coprocessor/AggregationClient.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/coprocessor/AggregationClient.java index 3b372382a26..9058289402e 100644 --- a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/coprocessor/AggregationClient.java +++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/coprocessor/AggregationClient.java @@ -40,6 +40,7 @@ import org.apache.hadoop.hbase.client.HTable; import org.apache.hadoop.hbase.client.Result; import org.apache.hadoop.hbase.client.ResultScanner; import org.apache.hadoop.hbase.client.Scan; +import org.apache.hadoop.hbase.client.Table; import org.apache.hadoop.hbase.coprocessor.ColumnInterpreter; import org.apache.hadoop.hbase.ipc.BlockingRpcCallback; import org.apache.hadoop.hbase.ipc.ServerRpcController; @@ -101,7 +102,7 @@ public class AggregationClient { public R max( final TableName tableName, final ColumnInterpreter ci, final Scan scan) throws Throwable { - HTable table = null; + Table table = null; try { table = new HTable(conf, tableName); return max(table, ci, scan); @@ -125,7 +126,7 @@ public class AggregationClient { * & propagated to it. */ public - R max(final HTable table, final ColumnInterpreter ci, + R max(final Table table, final ColumnInterpreter ci, final Scan scan) throws Throwable { final AggregateRequest requestArg = validateArgAndGetPB(scan, ci, false); class MaxCallBack implements Batch.Callback { @@ -196,7 +197,7 @@ public class AggregationClient { public R min( final TableName tableName, final ColumnInterpreter ci, final Scan scan) throws Throwable { - HTable table = null; + Table table = null; try { table = new HTable(conf, tableName); return min(table, ci, scan); @@ -218,7 +219,7 @@ public class AggregationClient { * @throws Throwable */ public - R min(final HTable table, final ColumnInterpreter ci, + R min(final Table table, final ColumnInterpreter ci, final Scan scan) throws Throwable { final AggregateRequest requestArg = validateArgAndGetPB(scan, ci, false); class MinCallBack implements Batch.Callback { @@ -276,7 +277,7 @@ public class AggregationClient { public long rowCount( final TableName tableName, final ColumnInterpreter ci, final Scan scan) throws Throwable { - HTable table = null; + Table table = null; try { table = new HTable(conf, tableName); return rowCount(table, ci, scan); @@ -301,7 +302,7 @@ public class AggregationClient { * @throws Throwable */ public - long rowCount(final HTable table, + long rowCount(final Table table, final ColumnInterpreter ci, final Scan scan) throws Throwable { final AggregateRequest requestArg = validateArgAndGetPB(scan, ci, true); class RowNumCallback implements Batch.Callback { @@ -350,7 +351,7 @@ public class AggregationClient { public S sum( final TableName tableName, final ColumnInterpreter ci, final Scan scan) throws Throwable { - HTable table = null; + Table table = null; try { table = new HTable(conf, tableName); return sum(table, ci, scan); @@ -371,7 +372,7 @@ public class AggregationClient { * @throws Throwable */ public - S sum(final HTable table, final ColumnInterpreter ci, + S sum(final Table table, final ColumnInterpreter ci, final Scan scan) throws Throwable { final AggregateRequest requestArg = validateArgAndGetPB(scan, ci, false); @@ -423,7 +424,7 @@ public class AggregationClient { private Pair getAvgArgs( final TableName tableName, final ColumnInterpreter ci, final Scan scan) throws Throwable { - HTable table = null; + Table table = null; try { table = new HTable(conf, tableName); return getAvgArgs(table, ci, scan); @@ -443,7 +444,7 @@ public class AggregationClient { * @throws Throwable */ private - Pair getAvgArgs(final HTable table, + Pair getAvgArgs(final Table table, final ColumnInterpreter ci, final Scan scan) throws Throwable { final AggregateRequest requestArg = validateArgAndGetPB(scan, ci, false); class AvgCallBack implements Batch.Callback> { @@ -523,7 +524,7 @@ public class AggregationClient { * @throws Throwable */ public double avg( - final HTable table, final ColumnInterpreter ci, Scan scan) throws Throwable { + final Table table, final ColumnInterpreter ci, Scan scan) throws Throwable { Pair p = getAvgArgs(table, ci, scan); return ci.divideForAvg(p.getFirst(), p.getSecond()); } @@ -540,7 +541,7 @@ public class AggregationClient { * @throws Throwable */ private - Pair, Long> getStdArgs(final HTable table, + Pair, Long> getStdArgs(final Table table, final ColumnInterpreter ci, final Scan scan) throws Throwable { final AggregateRequest requestArg = validateArgAndGetPB(scan, ci, false); class StdCallback implements Batch.Callback, Long>> { @@ -614,7 +615,7 @@ public class AggregationClient { public double std(final TableName tableName, ColumnInterpreter ci, Scan scan) throws Throwable { - HTable table = null; + Table table = null; try { table = new HTable(conf, tableName); return std(table, ci, scan); @@ -638,7 +639,7 @@ public class AggregationClient { * @throws Throwable */ public double std( - final HTable table, ColumnInterpreter ci, Scan scan) throws Throwable { + final Table table, ColumnInterpreter ci, Scan scan) throws Throwable { Pair, Long> p = getStdArgs(table, ci, scan); double res = 0d; double avg = ci.divideForAvg(p.getFirst().get(0), p.getSecond()); @@ -662,7 +663,7 @@ public class AggregationClient { */ private Pair>, List> - getMedianArgs(final HTable table, + getMedianArgs(final Table table, final ColumnInterpreter ci, final Scan scan) throws Throwable { final AggregateRequest requestArg = validateArgAndGetPB(scan, ci, false); final NavigableMap> map = @@ -727,7 +728,7 @@ public class AggregationClient { public R median(final TableName tableName, ColumnInterpreter ci, Scan scan) throws Throwable { - HTable table = null; + Table table = null; try { table = new HTable(conf, tableName); return median(table, ci, scan); @@ -749,7 +750,7 @@ public class AggregationClient { * @throws Throwable */ public - R median(final HTable table, ColumnInterpreter ci, + R median(final Table table, ColumnInterpreter ci, Scan scan) throws Throwable { Pair>, List> p = getMedianArgs(table, ci, scan); byte[] startRow = null; diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/coprocessor/SecureBulkLoadClient.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/coprocessor/SecureBulkLoadClient.java index 48986b1eb6a..c4ce86683d6 100644 --- a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/coprocessor/SecureBulkLoadClient.java +++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/coprocessor/SecureBulkLoadClient.java @@ -21,12 +21,12 @@ package org.apache.hadoop.hbase.client.coprocessor; import static org.apache.hadoop.hbase.HConstants.EMPTY_START_ROW; import static org.apache.hadoop.hbase.HConstants.LAST_ROW; +import org.apache.hadoop.hbase.client.Table; import org.apache.hadoop.hbase.util.ByteStringer; import org.apache.hadoop.classification.InterfaceAudience; import org.apache.hadoop.fs.Path; import org.apache.hadoop.hbase.TableName; -import org.apache.hadoop.hbase.client.HTable; import org.apache.hadoop.hbase.ipc.BlockingRpcCallback; import org.apache.hadoop.hbase.ipc.CoprocessorRpcChannel; import org.apache.hadoop.hbase.ipc.ServerRpcController; @@ -47,9 +47,9 @@ import java.util.List; */ @InterfaceAudience.Private public class SecureBulkLoadClient { - private HTable table; + private Table table; - public SecureBulkLoadClient(HTable table) { + public SecureBulkLoadClient(Table table) { this.table = table; } diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/security/access/AccessControlClient.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/security/access/AccessControlClient.java index 35c14126c3b..87db23d52bc 100644 --- a/hbase-client/src/main/java/org/apache/hadoop/hbase/security/access/AccessControlClient.java +++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/security/access/AccessControlClient.java @@ -32,8 +32,10 @@ import org.apache.hadoop.hbase.MasterNotRunningException; import org.apache.hadoop.hbase.NamespaceDescriptor; import org.apache.hadoop.hbase.TableName; import org.apache.hadoop.hbase.ZooKeeperConnectionException; +import org.apache.hadoop.hbase.client.Admin; import org.apache.hadoop.hbase.client.HBaseAdmin; import org.apache.hadoop.hbase.client.HTable; +import org.apache.hadoop.hbase.client.Table; import org.apache.hadoop.hbase.client.coprocessor.Batch; import org.apache.hadoop.hbase.ipc.BlockingRpcCallback; import org.apache.hadoop.hbase.ipc.CoprocessorRpcChannel; @@ -70,7 +72,7 @@ public class AccessControlClient { public static GrantResponse grant(Configuration conf, final TableName tableName, final String userName, final byte[] family, final byte[] qual, final AccessControlProtos.Permission.Action... actions) throws Throwable { - HTable ht = null; + Table ht = null; try { TableName aclTableName = TableName.valueOf(NamespaceDescriptor.SYSTEM_NAMESPACE_NAME_STR, "acl"); @@ -150,7 +152,7 @@ public class AccessControlClient { public static RevokeResponse revoke(Configuration conf, final String username, final TableName tableName, final byte[] family, final byte[] qualifier, final AccessControlProtos.Permission.Action... actions) throws Throwable { - HTable ht = null; + Table ht = null; try { TableName aclTableName = TableName.valueOf(NamespaceDescriptor.SYSTEM_NAMESPACE_NAME_STR, "acl"); @@ -211,8 +213,8 @@ public class AccessControlClient { public static List getUserPermissions(Configuration conf, String tableRegex) throws Throwable { List permList = new ArrayList(); - HTable ht = null; - HBaseAdmin ha = null; + Table ht = null; + Admin ha = null; try { TableName aclTableName = TableName.valueOf(NamespaceDescriptor.SYSTEM_NAMESPACE_NAME_STR, "acl"); diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/security/visibility/VisibilityClient.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/security/visibility/VisibilityClient.java index 8a17994be88..b1f9c915e12 100644 --- a/hbase-client/src/main/java/org/apache/hadoop/hbase/security/visibility/VisibilityClient.java +++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/security/visibility/VisibilityClient.java @@ -22,6 +22,7 @@ import static org.apache.hadoop.hbase.security.visibility.VisibilityConstants.LA import java.io.IOException; import java.util.Map; +import org.apache.hadoop.hbase.client.Table; import org.apache.hadoop.hbase.util.ByteStringer; import org.apache.hadoop.classification.InterfaceAudience; import org.apache.hadoop.classification.InterfaceStability; @@ -72,7 +73,7 @@ public class VisibilityClient { */ public static VisibilityLabelsResponse addLabels(Configuration conf, final String[] labels) throws Throwable { - HTable ht = null; + Table ht = null; try { ht = new HTable(conf, LABELS_TABLE_NAME.getName()); Batch.Call callable = @@ -126,7 +127,7 @@ public class VisibilityClient { * @throws Throwable */ public static GetAuthsResponse getAuths(Configuration conf, final String user) throws Throwable { - HTable ht = null; + Table ht = null; try { ht = new HTable(conf, LABELS_TABLE_NAME.getName()); Batch.Call callable = @@ -168,7 +169,7 @@ public class VisibilityClient { private static VisibilityLabelsResponse setOrClearAuths(Configuration conf, final String[] auths, final String user, final boolean setOrClear) throws IOException, ServiceException, Throwable { - HTable ht = null; + Table ht = null; try { ht = new HTable(conf, LABELS_TABLE_NAME.getName()); Batch.Call callable = diff --git a/hbase-client/src/test/java/org/apache/hadoop/hbase/client/TestClientNoCluster.java b/hbase-client/src/test/java/org/apache/hadoop/hbase/client/TestClientNoCluster.java index 9e878b468d4..df5e6932aac 100644 --- a/hbase-client/src/test/java/org/apache/hadoop/hbase/client/TestClientNoCluster.java +++ b/hbase-client/src/test/java/org/apache/hadoop/hbase/client/TestClientNoCluster.java @@ -150,7 +150,7 @@ public class TestClientNoCluster extends Configured implements Tool { Configuration localConfig = HBaseConfiguration.create(this.conf); // This override mocks up our exists/get call to throw a RegionServerStoppedException. localConfig.set("hbase.client.connection.impl", RpcTimeoutConnection.class.getName()); - HTable table = new HTable(localConfig, TableName.META_TABLE_NAME); + Table table = new HTable(localConfig, TableName.META_TABLE_NAME); Throwable t = null; LOG.info("Start"); try { @@ -187,7 +187,7 @@ public class TestClientNoCluster extends Configured implements Tool { // and it has expired. Otherwise, if this functionality is broke, all retries will be run -- // all ten of them -- and we'll get the RetriesExhaustedException exception. localConfig.setInt(HConstants.HBASE_CLIENT_META_OPERATION_TIMEOUT, pause - 1); - HTable table = new HTable(localConfig, TableName.META_TABLE_NAME); + Table table = new HTable(localConfig, TableName.META_TABLE_NAME); Throwable t = null; try { // An exists call turns into a get w/ a flag. @@ -219,7 +219,7 @@ public class TestClientNoCluster extends Configured implements Tool { // Go against meta else we will try to find first region for the table on construction which // means we'll have to do a bunch more mocking. Tests that go against meta only should be // good for a bit of testing. - HTable table = new HTable(this.conf, TableName.META_TABLE_NAME); + Table table = new HTable(this.conf, TableName.META_TABLE_NAME); ResultScanner scanner = table.getScanner(HConstants.CATALOG_FAMILY); try { Result result = null; @@ -239,7 +239,7 @@ public class TestClientNoCluster extends Configured implements Tool { // Go against meta else we will try to find first region for the table on construction which // means we'll have to do a bunch more mocking. Tests that go against meta only should be // good for a bit of testing. - HTable table = new HTable(this.conf, TableName.META_TABLE_NAME); + Table table = new HTable(this.conf, TableName.META_TABLE_NAME); ResultScanner scanner = table.getScanner(HConstants.CATALOG_FAMILY); try { Result result = null; @@ -700,7 +700,7 @@ public class TestClientNoCluster extends Configured implements Tool { * @throws IOException */ static void cycle(int id, final Configuration c, final HConnection sharedConnection) throws IOException { - HTableInterface table = sharedConnection.getTable(BIG_USER_TABLE); + Table table = sharedConnection.getTable(BIG_USER_TABLE); table.setAutoFlushTo(false); long namespaceSpan = c.getLong("hbase.test.namespace.span", 1000000); long startTime = System.currentTimeMillis(); diff --git a/hbase-client/src/test/java/org/apache/hadoop/hbase/client/TestSnapshotFromAdmin.java b/hbase-client/src/test/java/org/apache/hadoop/hbase/client/TestSnapshotFromAdmin.java index 161946726c5..572128ac4f7 100644 --- a/hbase-client/src/test/java/org/apache/hadoop/hbase/client/TestSnapshotFromAdmin.java +++ b/hbase-client/src/test/java/org/apache/hadoop/hbase/client/TestSnapshotFromAdmin.java @@ -99,7 +99,7 @@ public class TestSnapshotFromAdmin { builder.build(), builder.build(), builder.build(), builder.setDone(true).build()); // setup the admin and run the test - HBaseAdmin admin = new HBaseAdmin(mockConnection); + Admin admin = new HBaseAdmin(mockConnection); String snapshot = "snapshot"; TableName table = TableName.valueOf("table"); // get start time @@ -122,7 +122,7 @@ public class TestSnapshotFromAdmin { .mock(ConnectionManager.HConnectionImplementation.class); Configuration conf = HBaseConfiguration.create(); Mockito.when(mockConnection.getConfiguration()).thenReturn(conf); - HBaseAdmin admin = new HBaseAdmin(mockConnection); + Admin admin = new HBaseAdmin(mockConnection); SnapshotDescription.Builder builder = SnapshotDescription.newBuilder(); // check that invalid snapshot names fail failSnapshotStart(admin, builder.setName(HConstants.SNAPSHOT_DIR_NAME).build()); @@ -152,7 +152,7 @@ public class TestSnapshotFromAdmin { admin.snapshot(builder.setName("snapshot").setTable("table").build()); } - private void failSnapshotStart(HBaseAdmin admin, SnapshotDescription snapshot) throws IOException { + private void failSnapshotStart(Admin admin, SnapshotDescription snapshot) throws IOException { try { admin.snapshot(snapshot); fail("Snapshot should not have succeed with name:" + snapshot.getName()); diff --git a/hbase-examples/src/test/java/org/apache/hadoop/hbase/coprocessor/example/TestBulkDeleteProtocol.java b/hbase-examples/src/test/java/org/apache/hadoop/hbase/coprocessor/example/TestBulkDeleteProtocol.java index 8dbb16c9531..16327ba5f38 100644 --- a/hbase-examples/src/test/java/org/apache/hadoop/hbase/coprocessor/example/TestBulkDeleteProtocol.java +++ b/hbase-examples/src/test/java/org/apache/hadoop/hbase/coprocessor/example/TestBulkDeleteProtocol.java @@ -37,6 +37,7 @@ import org.apache.hadoop.hbase.client.HTable; import org.apache.hadoop.hbase.client.Put; import org.apache.hadoop.hbase.client.Result; import org.apache.hadoop.hbase.client.Scan; +import org.apache.hadoop.hbase.client.Table; import org.apache.hadoop.hbase.client.coprocessor.Batch; import org.apache.hadoop.hbase.coprocessor.CoprocessorHost; import org.apache.hadoop.hbase.coprocessor.example.generated.BulkDeleteProtos.BulkDeleteRequest; @@ -78,7 +79,7 @@ public class TestBulkDeleteProtocol { // @Ignore @Test public void testBulkDeleteEndpoint() throws Throwable { byte[] tableName = Bytes.toBytes("testBulkDeleteEndpoint"); - HTable ht = createTable(tableName); + Table ht = createTable(tableName); List puts = new ArrayList(100); for (int j = 0; j < 100; j++) { byte[] rowkey = Bytes.toBytes(j); @@ -102,7 +103,7 @@ public class TestBulkDeleteProtocol { throws Throwable { byte[] tableName = Bytes .toBytes("testBulkDeleteEndpointWhenRowBatchSizeLessThanRowsToDeleteFromARegion"); - HTable ht = createTable(tableName); + Table ht = createTable(tableName); List puts = new ArrayList(100); for (int j = 0; j < 100; j++) { byte[] rowkey = Bytes.toBytes(j); @@ -123,7 +124,7 @@ public class TestBulkDeleteProtocol { private long invokeBulkDeleteProtocol(byte[] tableName, final Scan scan, final int rowBatchSize, final DeleteType deleteType, final Long timeStamp) throws Throwable { - HTable ht = new HTable(TEST_UTIL.getConfiguration(), tableName); + Table ht = new HTable(TEST_UTIL.getConfiguration(), tableName); long noOfDeletedRows = 0L; Batch.Call callable = new Batch.Call() { @@ -155,7 +156,7 @@ public class TestBulkDeleteProtocol { // @Ignore @Test public void testBulkDeleteWithConditionBasedDelete() throws Throwable { byte[] tableName = Bytes.toBytes("testBulkDeleteWithConditionBasedDelete"); - HTable ht = createTable(tableName); + Table ht = createTable(tableName); List puts = new ArrayList(100); for (int j = 0; j < 100; j++) { byte[] rowkey = Bytes.toBytes(j); @@ -185,7 +186,7 @@ public class TestBulkDeleteProtocol { // @Ignore @Test public void testBulkDeleteColumn() throws Throwable { byte[] tableName = Bytes.toBytes("testBulkDeleteColumn"); - HTable ht = createTable(tableName); + Table ht = createTable(tableName); List puts = new ArrayList(100); for (int j = 0; j < 100; j++) { byte[] rowkey = Bytes.toBytes(j); @@ -218,7 +219,7 @@ public class TestBulkDeleteProtocol { htd.addFamily(new HColumnDescriptor(FAMILY1)); htd.addFamily(new HColumnDescriptor(FAMILY2)); TEST_UTIL.getHBaseAdmin().createTable(htd, Bytes.toBytes(0), Bytes.toBytes(120), 5); - HTable ht = new HTable(TEST_UTIL.getConfiguration(), tableName); + Table ht = new HTable(TEST_UTIL.getConfiguration(), tableName); List puts = new ArrayList(100); for (int j = 0; j < 100; j++) { Put put = new Put(Bytes.toBytes(j)); @@ -245,7 +246,7 @@ public class TestBulkDeleteProtocol { // @Ignore @Test public void testBulkDeleteColumnVersion() throws Throwable { byte[] tableName = Bytes.toBytes("testBulkDeleteColumnVersion"); - HTable ht = createTable(tableName); + Table ht = createTable(tableName); List puts = new ArrayList(100); for (int j = 0; j < 100; j++) { Put put = new Put(Bytes.toBytes(j)); @@ -293,7 +294,7 @@ public class TestBulkDeleteProtocol { // @Ignore @Test public void testBulkDeleteColumnVersionBasedOnTS() throws Throwable { byte[] tableName = Bytes.toBytes("testBulkDeleteColumnVersionBasedOnTS"); - HTable ht = createTable(tableName); + Table ht = createTable(tableName); List puts = new ArrayList(100); for (int j = 0; j < 100; j++) { Put put = new Put(Bytes.toBytes(j)); @@ -340,7 +341,7 @@ public class TestBulkDeleteProtocol { // @Ignore @Test public void testBulkDeleteWithNumberOfVersions() throws Throwable { byte[] tableName = Bytes.toBytes("testBulkDeleteWithNumberOfVersions"); - HTable ht = createTable(tableName); + Table ht = createTable(tableName); List puts = new ArrayList(100); for (int j = 0; j < 100; j++) { Put put = new Put(Bytes.toBytes(j)); @@ -422,13 +423,13 @@ public class TestBulkDeleteProtocol { ht.close(); } - private HTable createTable(byte[] tableName) throws IOException { + private Table createTable(byte[] tableName) throws IOException { HTableDescriptor htd = new HTableDescriptor(TableName.valueOf(tableName)); HColumnDescriptor hcd = new HColumnDescriptor(FAMILY1); hcd.setMaxVersions(10);// Just setting 10 as I am not testing with more than 10 versions here htd.addFamily(hcd); TEST_UTIL.getHBaseAdmin().createTable(htd, Bytes.toBytes(0), Bytes.toBytes(120), 5); - HTable ht = new HTable(TEST_UTIL.getConfiguration(), tableName); + Table ht = new HTable(TEST_UTIL.getConfiguration(), tableName); return ht; } diff --git a/hbase-examples/src/test/java/org/apache/hadoop/hbase/coprocessor/example/TestRowCountEndpoint.java b/hbase-examples/src/test/java/org/apache/hadoop/hbase/coprocessor/example/TestRowCountEndpoint.java index fb04b4d881b..cc1648315f0 100644 --- a/hbase-examples/src/test/java/org/apache/hadoop/hbase/coprocessor/example/TestRowCountEndpoint.java +++ b/hbase-examples/src/test/java/org/apache/hadoop/hbase/coprocessor/example/TestRowCountEndpoint.java @@ -23,16 +23,13 @@ import org.apache.hadoop.hbase.HBaseTestingUtility; import org.apache.hadoop.hbase.MediumTests; import org.apache.hadoop.hbase.client.HTable; import org.apache.hadoop.hbase.client.Put; +import org.apache.hadoop.hbase.client.Table; import org.apache.hadoop.hbase.client.coprocessor.Batch; import org.apache.hadoop.hbase.coprocessor.CoprocessorHost; import org.apache.hadoop.hbase.coprocessor.example.generated.ExampleProtos; import org.apache.hadoop.hbase.ipc.BlockingRpcCallback; import org.apache.hadoop.hbase.ipc.ServerRpcController; import org.apache.hadoop.hbase.util.Bytes; -import org.junit.AfterClass; -import org.junit.BeforeClass; -import org.junit.Test; -import org.junit.Ignore; import org.junit.experimental.categories.Category; import java.io.IOException; @@ -72,7 +69,7 @@ public class TestRowCountEndpoint { // @Ignore @Test public void testEndpoint() throws Throwable { - HTable table = new HTable(CONF, TEST_TABLE); + Table table = new HTable(CONF, TEST_TABLE); // insert some test rows for (int i=0; i<5; i++) { diff --git a/hbase-examples/src/test/java/org/apache/hadoop/hbase/coprocessor/example/TestZooKeeperScanPolicyObserver.java b/hbase-examples/src/test/java/org/apache/hadoop/hbase/coprocessor/example/TestZooKeeperScanPolicyObserver.java index 824910aff8c..af51504c125 100644 --- a/hbase-examples/src/test/java/org/apache/hadoop/hbase/coprocessor/example/TestZooKeeperScanPolicyObserver.java +++ b/hbase-examples/src/test/java/org/apache/hadoop/hbase/coprocessor/example/TestZooKeeperScanPolicyObserver.java @@ -32,6 +32,7 @@ import org.apache.hadoop.hbase.client.Get; import org.apache.hadoop.hbase.client.HTable; import org.apache.hadoop.hbase.client.Put; import org.apache.hadoop.hbase.client.Result; +import org.apache.hadoop.hbase.client.Table; import org.apache.hadoop.hbase.coprocessor.CoprocessorHost; import org.apache.hadoop.hbase.util.Bytes; import org.apache.hadoop.hbase.util.EnvironmentEdgeManager; @@ -74,7 +75,7 @@ public class TestZooKeeperScanPolicyObserver { .setTimeToLive(1); desc.addFamily(hcd); TEST_UTIL.getHBaseAdmin().createTable(desc); - HTable t = new HTable(new Configuration(TEST_UTIL.getConfiguration()), tableName); + Table t = new HTable(new Configuration(TEST_UTIL.getConfiguration()), tableName); long now = EnvironmentEdgeManager.currentTime(); ZooKeeperWatcher zkw = new ZooKeeperWatcher(TEST_UTIL.getConfiguration(), "test", null); diff --git a/hbase-it/src/test/java/org/apache/hadoop/hbase/DistributedHBaseCluster.java b/hbase-it/src/test/java/org/apache/hadoop/hbase/DistributedHBaseCluster.java index 9ae00f9d9cf..f1f5f93949d 100644 --- a/hbase-it/src/test/java/org/apache/hadoop/hbase/DistributedHBaseCluster.java +++ b/hbase-it/src/test/java/org/apache/hadoop/hbase/DistributedHBaseCluster.java @@ -25,6 +25,7 @@ import java.util.List; import org.apache.hadoop.classification.InterfaceAudience; import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.hbase.ClusterManager.ServiceType; +import org.apache.hadoop.hbase.client.Admin; import org.apache.hadoop.hbase.client.HBaseAdmin; import org.apache.hadoop.hbase.client.HConnection; import org.apache.hadoop.hbase.client.HConnectionManager; @@ -45,7 +46,7 @@ import com.google.common.collect.Sets; @InterfaceAudience.Private public class DistributedHBaseCluster extends HBaseCluster { - private HBaseAdmin admin; + private Admin admin; private ClusterManager clusterManager; diff --git a/hbase-it/src/test/java/org/apache/hadoop/hbase/IntegrationTestLazyCfLoading.java b/hbase-it/src/test/java/org/apache/hadoop/hbase/IntegrationTestLazyCfLoading.java index a1e306d707e..750376a9961 100644 --- a/hbase-it/src/test/java/org/apache/hadoop/hbase/IntegrationTestLazyCfLoading.java +++ b/hbase-it/src/test/java/org/apache/hadoop/hbase/IntegrationTestLazyCfLoading.java @@ -31,6 +31,7 @@ import org.apache.hadoop.hbase.client.HTable; import org.apache.hadoop.hbase.client.Result; import org.apache.hadoop.hbase.client.ResultScanner; import org.apache.hadoop.hbase.client.Scan; +import org.apache.hadoop.hbase.client.Table; import org.apache.hadoop.hbase.filter.CompareFilter; import org.apache.hadoop.hbase.filter.Filter; import org.apache.hadoop.hbase.filter.SingleColumnValueFilter; @@ -222,7 +223,7 @@ public class IntegrationTestLazyCfLoading { long maxRuntime = conf.getLong(timeoutKey, DEFAULT_TIMEOUT_MINUTES); long serverCount = util.getHBaseClusterInterface().getClusterStatus().getServersSize(); long keysToWrite = serverCount * KEYS_TO_WRITE_PER_SERVER; - HTable table = new HTable(conf, TABLE_NAME); + Table table = new HTable(conf, TABLE_NAME); // Create multi-threaded writer and start it. We write multiple columns/CFs and verify // their integrity, therefore multi-put is necessary. diff --git a/hbase-it/src/test/java/org/apache/hadoop/hbase/mapreduce/IntegrationTestImportTsv.java b/hbase-it/src/test/java/org/apache/hadoop/hbase/mapreduce/IntegrationTestImportTsv.java index 766c66ff4f2..e99677ba5d1 100644 --- a/hbase-it/src/test/java/org/apache/hadoop/hbase/mapreduce/IntegrationTestImportTsv.java +++ b/hbase-it/src/test/java/org/apache/hadoop/hbase/mapreduce/IntegrationTestImportTsv.java @@ -48,6 +48,7 @@ import org.apache.hadoop.hbase.TableName; import org.apache.hadoop.hbase.client.HTable; import org.apache.hadoop.hbase.client.Result; import org.apache.hadoop.hbase.client.Scan; +import org.apache.hadoop.hbase.client.Table; import org.apache.hadoop.hbase.util.Bytes; import org.apache.hadoop.io.LongWritable; import org.apache.hadoop.io.Text; @@ -153,7 +154,7 @@ public class IntegrationTestImportTsv implements Configurable, Tool { assertEquals("Loading HFiles failed.", 0, ToolRunner.run(new LoadIncrementalHFiles(new Configuration(getConf())), args)); - HTable table = null; + Table table = null; Scan scan = new Scan() {{ setCacheBlocks(false); setCaching(1000); diff --git a/hbase-it/src/test/java/org/apache/hadoop/hbase/mttr/IntegrationTestMTTR.java b/hbase-it/src/test/java/org/apache/hadoop/hbase/mttr/IntegrationTestMTTR.java index 65e1026cec3..55cc70eec50 100644 --- a/hbase-it/src/test/java/org/apache/hadoop/hbase/mttr/IntegrationTestMTTR.java +++ b/hbase-it/src/test/java/org/apache/hadoop/hbase/mttr/IntegrationTestMTTR.java @@ -49,6 +49,7 @@ import org.apache.hadoop.hbase.chaos.actions.RestartActiveMasterAction; import org.apache.hadoop.hbase.chaos.actions.RestartRsHoldingMetaAction; import org.apache.hadoop.hbase.chaos.actions.RestartRsHoldingTableAction; import org.apache.hadoop.hbase.chaos.factories.MonkeyConstants; +import org.apache.hadoop.hbase.client.Admin; import org.apache.hadoop.hbase.client.HBaseAdmin; import org.apache.hadoop.hbase.client.HTable; import org.apache.hadoop.hbase.client.Put; @@ -56,6 +57,7 @@ import org.apache.hadoop.hbase.client.Result; import org.apache.hadoop.hbase.client.ResultScanner; import org.apache.hadoop.hbase.client.RetriesExhaustedException; import org.apache.hadoop.hbase.client.Scan; +import org.apache.hadoop.hbase.client.Table; import org.apache.hadoop.hbase.coprocessor.CoprocessorException; import org.apache.hadoop.hbase.filter.KeyOnlyFilter; import org.apache.hadoop.hbase.ipc.FatalConnectionException; @@ -461,7 +463,7 @@ public class IntegrationTestMTTR { */ static class PutCallable extends TimingCallable { - private final HTable table; + private final Table table; public PutCallable(Future f) throws IOException { super(f); @@ -488,7 +490,7 @@ public class IntegrationTestMTTR { * supplied future returns. Returns the max time taken to scan. */ static class ScanCallable extends TimingCallable { - private final HTable table; + private final Table table; public ScanCallable(Future f) throws IOException { super(f); @@ -531,7 +533,7 @@ public class IntegrationTestMTTR { @Override protected boolean doAction() throws Exception { - HBaseAdmin admin = null; + Admin admin = null; try { admin = new HBaseAdmin(util.getConfiguration()); ClusterStatus status = admin.getClusterStatus(); diff --git a/hbase-it/src/test/java/org/apache/hadoop/hbase/test/IntegrationTestBigLinkedList.java b/hbase-it/src/test/java/org/apache/hadoop/hbase/test/IntegrationTestBigLinkedList.java index 3adef2616cc..c709f0d94c9 100644 --- a/hbase-it/src/test/java/org/apache/hadoop/hbase/test/IntegrationTestBigLinkedList.java +++ b/hbase-it/src/test/java/org/apache/hadoop/hbase/test/IntegrationTestBigLinkedList.java @@ -50,6 +50,7 @@ import org.apache.hadoop.hbase.IntegrationTestingUtility; import org.apache.hadoop.hbase.IntegrationTests; import org.apache.hadoop.hbase.MasterNotRunningException; import org.apache.hadoop.hbase.TableName; +import org.apache.hadoop.hbase.client.Admin; import org.apache.hadoop.hbase.client.Get; import org.apache.hadoop.hbase.client.HBaseAdmin; import org.apache.hadoop.hbase.client.HConnection; @@ -60,6 +61,7 @@ import org.apache.hadoop.hbase.client.Result; import org.apache.hadoop.hbase.client.ResultScanner; import org.apache.hadoop.hbase.client.Scan; import org.apache.hadoop.hbase.client.ScannerCallable; +import org.apache.hadoop.hbase.client.Table; import org.apache.hadoop.hbase.io.ImmutableBytesWritable; import org.apache.hadoop.hbase.mapreduce.TableMapReduceUtil; import org.apache.hadoop.hbase.mapreduce.TableMapper; @@ -448,7 +450,7 @@ public class IntegrationTestBigLinkedList extends IntegrationTestBase { protected void createSchema() throws IOException { Configuration conf = getConf(); - HBaseAdmin admin = new HBaseAdmin(conf); + Admin admin = new HBaseAdmin(conf); TableName tableName = getTableName(conf); try { if (!admin.tableExists(tableName)) { @@ -873,7 +875,7 @@ public class IntegrationTestBigLinkedList extends IntegrationTestBase { System.exit(-1); } - HTable table = new HTable(getConf(), getTableName(getConf())); + Table table = new HTable(getConf(), getTableName(getConf())); Scan scan = new Scan(); scan.setBatch(10000); @@ -923,7 +925,7 @@ public class IntegrationTestBigLinkedList extends IntegrationTestBase { org.apache.hadoop.hbase.client.Delete delete = new org.apache.hadoop.hbase.client.Delete(val); - HTable table = new HTable(getConf(), getTableName(getConf())); + Table table = new HTable(getConf(), getTableName(getConf())); table.delete(delete); table.flushCommits(); @@ -969,7 +971,7 @@ public class IntegrationTestBigLinkedList extends IntegrationTestBase { byte[] startKey = isSpecificStart ? Bytes.toBytesBinary(cmd.getOptionValue('s')) : null; int logEvery = cmd.hasOption('l') ? Integer.parseInt(cmd.getOptionValue('l')) : 1; - HTable table = new HTable(getConf(), getTableName(getConf())); + Table table = new HTable(getConf(), getTableName(getConf())); long numQueries = 0; // If isSpecificStart is set, only walk one list from that particular node. // Note that in case of circular (or P-shaped) list it will walk forever, as is @@ -1005,7 +1007,7 @@ public class IntegrationTestBigLinkedList extends IntegrationTestBase { return 0; } - private static CINode findStartNode(HTable table, byte[] startKey) throws IOException { + private static CINode findStartNode(Table table, byte[] startKey) throws IOException { Scan scan = new Scan(); scan.setStartRow(startKey); scan.setBatch(1); @@ -1028,7 +1030,7 @@ public class IntegrationTestBigLinkedList extends IntegrationTestBase { return null; } - private CINode getNode(byte[] row, HTable table, CINode node) throws IOException { + private CINode getNode(byte[] row, Table table, CINode node) throws IOException { Get get = new Get(row); get.addColumn(FAMILY_NAME, COLUMN_PREV); Result result = table.get(get); diff --git a/hbase-it/src/test/java/org/apache/hadoop/hbase/test/IntegrationTestBigLinkedListWithVisibility.java b/hbase-it/src/test/java/org/apache/hadoop/hbase/test/IntegrationTestBigLinkedListWithVisibility.java index 9748b3142ac..be7e36f9255 100644 --- a/hbase-it/src/test/java/org/apache/hadoop/hbase/test/IntegrationTestBigLinkedListWithVisibility.java +++ b/hbase-it/src/test/java/org/apache/hadoop/hbase/test/IntegrationTestBigLinkedListWithVisibility.java @@ -38,6 +38,7 @@ import org.apache.hadoop.hbase.IntegrationTestingUtility; import org.apache.hadoop.hbase.IntegrationTests; import org.apache.hadoop.hbase.TableName; import org.apache.hadoop.hbase.chaos.factories.MonkeyFactory; +import org.apache.hadoop.hbase.client.Admin; import org.apache.hadoop.hbase.client.Delete; import org.apache.hadoop.hbase.client.HBaseAdmin; import org.apache.hadoop.hbase.client.HConnection; @@ -46,6 +47,7 @@ import org.apache.hadoop.hbase.client.HTable; import org.apache.hadoop.hbase.client.Put; import org.apache.hadoop.hbase.client.Result; import org.apache.hadoop.hbase.client.Scan; +import org.apache.hadoop.hbase.client.Table; import org.apache.hadoop.hbase.io.ImmutableBytesWritable; import org.apache.hadoop.hbase.io.hfile.HFile; import org.apache.hadoop.hbase.mapreduce.Import; @@ -130,7 +132,7 @@ public class IntegrationTestBigLinkedListWithVisibility extends IntegrationTestB if(!acl) { LOG.info("No ACL available."); } - HBaseAdmin admin = new HBaseAdmin(getConf()); + Admin admin = new HBaseAdmin(getConf()); for (int i = 0; i < DEFAULT_TABLES_COUNT; i++) { TableName tableName = IntegrationTestBigLinkedListWithVisibility.getTableName(i); createTable(admin, tableName, false, acl); @@ -140,7 +142,7 @@ public class IntegrationTestBigLinkedListWithVisibility extends IntegrationTestB admin.close(); } - private void createTable(HBaseAdmin admin, TableName tableName, boolean setVersion, + private void createTable(Admin admin, TableName tableName, boolean setVersion, boolean acl) throws IOException { if (!admin.tableExists(tableName)) { HTableDescriptor htd = new HTableDescriptor(tableName); @@ -170,8 +172,8 @@ public class IntegrationTestBigLinkedListWithVisibility extends IntegrationTestB } static class VisibilityGeneratorMapper extends GeneratorMapper { - HTable[] tables = new HTable[DEFAULT_TABLES_COUNT]; - HTable commonTable = null; + Table[] tables = new Table[DEFAULT_TABLES_COUNT]; + Table commonTable = null; @Override protected void setup(org.apache.hadoop.mapreduce.Mapper.Context context) throws IOException, diff --git a/hbase-it/src/test/java/org/apache/hadoop/hbase/test/IntegrationTestLoadAndVerify.java b/hbase-it/src/test/java/org/apache/hadoop/hbase/test/IntegrationTestLoadAndVerify.java index 0da510745eb..6bab237ad9c 100644 --- a/hbase-it/src/test/java/org/apache/hadoop/hbase/test/IntegrationTestLoadAndVerify.java +++ b/hbase-it/src/test/java/org/apache/hadoop/hbase/test/IntegrationTestLoadAndVerify.java @@ -448,7 +448,7 @@ public void cleanUpCluster() throws Exception { HTableDescriptor htd = new HTableDescriptor(table); htd.addFamily(new HColumnDescriptor(TEST_FAMILY)); - HBaseAdmin admin = new HBaseAdmin(getConf()); + Admin admin = new HBaseAdmin(getConf()); if (doLoad) { admin.createTable(htd, Bytes.toBytes(0L), Bytes.toBytes(-1L), numPresplits); doLoad(getConf(), htd); diff --git a/hbase-it/src/test/java/org/apache/hadoop/hbase/test/IntegrationTestTimeBoundedRequestsWithRegionReplicas.java b/hbase-it/src/test/java/org/apache/hadoop/hbase/test/IntegrationTestTimeBoundedRequestsWithRegionReplicas.java index 5ca0e362e13..63bd42f04be 100644 --- a/hbase-it/src/test/java/org/apache/hadoop/hbase/test/IntegrationTestTimeBoundedRequestsWithRegionReplicas.java +++ b/hbase-it/src/test/java/org/apache/hadoop/hbase/test/IntegrationTestTimeBoundedRequestsWithRegionReplicas.java @@ -39,8 +39,8 @@ import org.apache.hadoop.hbase.chaos.factories.MonkeyFactory; import org.apache.hadoop.hbase.client.Admin; import org.apache.hadoop.hbase.client.Consistency; import org.apache.hadoop.hbase.client.Get; -import org.apache.hadoop.hbase.client.HTableInterface; import org.apache.hadoop.hbase.client.Result; +import org.apache.hadoop.hbase.client.Table; import org.apache.hadoop.hbase.regionserver.StorefileRefresherChore; import org.apache.hadoop.hbase.util.LoadTestTool; import org.apache.hadoop.hbase.util.MultiThreadedReader; @@ -326,7 +326,7 @@ public class IntegrationTestTimeBoundedRequestsWithRegionReplicas extends Integr @Override protected void verifyResultsAndUpdateMetrics(boolean verify, Get[] gets, long elapsedNano, - Result[] results, HTableInterface table, boolean isNullExpected) + Result[] results, Table table, boolean isNullExpected) throws IOException { super.verifyResultsAndUpdateMetrics(verify, gets, elapsedNano, results, table, isNullExpected); for (Result r : results) { diff --git a/hbase-it/src/test/java/org/apache/hadoop/hbase/test/IntegrationTestWithCellVisibilityLoadAndVerify.java b/hbase-it/src/test/java/org/apache/hadoop/hbase/test/IntegrationTestWithCellVisibilityLoadAndVerify.java index f4e42501ce6..fc3ae3c3e7d 100644 --- a/hbase-it/src/test/java/org/apache/hadoop/hbase/test/IntegrationTestWithCellVisibilityLoadAndVerify.java +++ b/hbase-it/src/test/java/org/apache/hadoop/hbase/test/IntegrationTestWithCellVisibilityLoadAndVerify.java @@ -33,6 +33,7 @@ import org.apache.hadoop.hbase.HConstants; import org.apache.hadoop.hbase.HTableDescriptor; import org.apache.hadoop.hbase.IntegrationTestingUtility; import org.apache.hadoop.hbase.IntegrationTests; +import org.apache.hadoop.hbase.client.Admin; import org.apache.hadoop.hbase.client.HBaseAdmin; import org.apache.hadoop.hbase.client.Put; import org.apache.hadoop.hbase.client.Result; @@ -369,7 +370,7 @@ public class IntegrationTestWithCellVisibilityLoadAndVerify extends IntegrationT HTableDescriptor htd = new HTableDescriptor(getTablename()); htd.addFamily(new HColumnDescriptor(TEST_FAMILY)); - HBaseAdmin admin = new HBaseAdmin(getConf()); + Admin admin = new HBaseAdmin(getConf()); try { admin.createTable(htd, Bytes.toBytes(0L), Bytes.toBytes(-1L), numPresplits); } finally { diff --git a/hbase-it/src/test/java/org/apache/hadoop/hbase/trace/IntegrationTestSendTraceRequests.java b/hbase-it/src/test/java/org/apache/hadoop/hbase/trace/IntegrationTestSendTraceRequests.java index 7961eb9c939..2ec5838cb11 100644 --- a/hbase-it/src/test/java/org/apache/hadoop/hbase/trace/IntegrationTestSendTraceRequests.java +++ b/hbase-it/src/test/java/org/apache/hadoop/hbase/trace/IntegrationTestSendTraceRequests.java @@ -31,6 +31,7 @@ import org.apache.hadoop.hbase.client.Put; import org.apache.hadoop.hbase.client.Result; import org.apache.hadoop.hbase.client.ResultScanner; import org.apache.hadoop.hbase.client.Scan; +import org.apache.hadoop.hbase.client.Table; import org.apache.hadoop.hbase.util.AbstractHBaseTool; import org.apache.hadoop.hbase.util.Bytes; import org.apache.hadoop.util.ToolRunner; @@ -123,7 +124,7 @@ public class IntegrationTestSendTraceRequests extends AbstractHBaseTool { ResultScanner rs = null; try { innerScope = Trace.startSpan("Scan", Sampler.ALWAYS); - HTable ht = new HTable(util.getConfiguration(), tableName); + Table ht = new HTable(util.getConfiguration(), tableName); Scan s = new Scan(); s.setStartRow(Bytes.toBytes(rowKeyQueue.take())); s.setBatch(7); @@ -171,7 +172,7 @@ public class IntegrationTestSendTraceRequests extends AbstractHBaseTool { public void run() { - HTable ht = null; + Table ht = null; try { ht = new HTable(util.getConfiguration(), tableName); } catch (IOException e) { diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/LocalHBaseCluster.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/LocalHBaseCluster.java index a30819dd284..d4a87d31960 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/LocalHBaseCluster.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/LocalHBaseCluster.java @@ -29,6 +29,7 @@ import org.apache.commons.logging.LogFactory; import org.apache.hadoop.classification.InterfaceAudience; import org.apache.hadoop.classification.InterfaceStability; import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.hbase.client.Admin; import org.apache.hadoop.hbase.client.HBaseAdmin; import org.apache.hadoop.hbase.regionserver.HRegionServer; import org.apache.hadoop.hbase.security.User; @@ -462,7 +463,7 @@ public class LocalHBaseCluster { Configuration conf = HBaseConfiguration.create(); LocalHBaseCluster cluster = new LocalHBaseCluster(conf); cluster.startup(); - HBaseAdmin admin = new HBaseAdmin(conf); + Admin admin = new HBaseAdmin(conf); HTableDescriptor htd = new HTableDescriptor(TableName.valueOf(cluster.getClass().getName())); admin.createTable(htd); diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/coprocessor/CoprocessorHost.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/coprocessor/CoprocessorHost.java index ab76cd4cead..155990eb954 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/coprocessor/CoprocessorHost.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/coprocessor/CoprocessorHost.java @@ -47,6 +47,7 @@ import org.apache.hadoop.hbase.TableName; import org.apache.hadoop.hbase.client.HTable; import org.apache.hadoop.hbase.client.HTableInterface; import org.apache.hadoop.hbase.client.HTableWrapper; +import org.apache.hadoop.hbase.client.Table; import org.apache.hadoop.hbase.util.Bytes; import org.apache.hadoop.hbase.util.CoprocessorClassLoader; import org.apache.hadoop.hbase.util.SortedCopyOnWriteSet; diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/mapred/HRegionPartitioner.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/mapred/HRegionPartitioner.java index b6419b81aa1..3af2351d6a0 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/mapred/HRegionPartitioner.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/mapred/HRegionPartitioner.java @@ -26,6 +26,7 @@ import org.apache.hadoop.classification.InterfaceAudience; import org.apache.hadoop.classification.InterfaceStability; import org.apache.hadoop.hbase.HBaseConfiguration; import org.apache.hadoop.hbase.client.HTable; +import org.apache.hadoop.hbase.client.RegionLocator; import org.apache.hadoop.hbase.io.ImmutableBytesWritable; import org.apache.hadoop.hbase.util.Bytes; import org.apache.hadoop.mapred.JobConf; @@ -46,7 +47,7 @@ import org.apache.hadoop.mapred.Partitioner; public class HRegionPartitioner implements Partitioner { private static final Log LOG = LogFactory.getLog(HRegionPartitioner.class); - private HTable table; + private RegionLocator table; private byte[][] startKeys; public void configure(JobConf job) { diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/mapred/TableOutputFormat.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/mapred/TableOutputFormat.java index 327e4045350..e9d3932f779 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/mapred/TableOutputFormat.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/mapred/TableOutputFormat.java @@ -28,6 +28,7 @@ import org.apache.hadoop.fs.FileSystem; import org.apache.hadoop.hbase.HBaseConfiguration; import org.apache.hadoop.hbase.client.HTable; import org.apache.hadoop.hbase.client.Put; +import org.apache.hadoop.hbase.client.Table; import org.apache.hadoop.hbase.io.ImmutableBytesWritable; import org.apache.hadoop.fs.FileAlreadyExistsException; import org.apache.hadoop.mapred.InvalidJobConfException; @@ -56,14 +57,14 @@ FileOutputFormat { */ protected static class TableRecordWriter implements RecordWriter { - private HTable m_table; + private Table m_table; /** * Instantiate a TableRecordWriter with the HBase HClient for writing. * * @param table */ - public TableRecordWriter(HTable table) { + public TableRecordWriter(Table table) { m_table = table; } diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/mapred/TableRecordReader.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/mapred/TableRecordReader.java index 7713180dd73..a9496a1b9e0 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/mapred/TableRecordReader.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/mapred/TableRecordReader.java @@ -24,6 +24,7 @@ import org.apache.hadoop.classification.InterfaceAudience; import org.apache.hadoop.classification.InterfaceStability; import org.apache.hadoop.hbase.client.HTable; import org.apache.hadoop.hbase.client.Result; +import org.apache.hadoop.hbase.client.Table; import org.apache.hadoop.hbase.filter.Filter; import org.apache.hadoop.hbase.io.ImmutableBytesWritable; import org.apache.hadoop.mapred.RecordReader; @@ -62,7 +63,7 @@ implements RecordReader { /** * @param htable the {@link HTable} to scan. */ - public void setHTable(HTable htable) { + public void setHTable(Table htable) { this.recordReaderImpl.setHTable(htable); } diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/mapred/TableRecordReaderImpl.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/mapred/TableRecordReaderImpl.java index f4043f4836f..7517c1f56e4 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/mapred/TableRecordReaderImpl.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/mapred/TableRecordReaderImpl.java @@ -30,6 +30,7 @@ import org.apache.hadoop.hbase.client.Result; import org.apache.hadoop.hbase.client.ResultScanner; import org.apache.hadoop.hbase.client.Scan; import org.apache.hadoop.hbase.client.ScannerCallable; +import org.apache.hadoop.hbase.client.Table; import org.apache.hadoop.hbase.filter.Filter; import org.apache.hadoop.hbase.io.ImmutableBytesWritable; import org.apache.hadoop.hbase.mapreduce.TableInputFormat; @@ -52,7 +53,7 @@ public class TableRecordReaderImpl { private byte [] lastSuccessfulRow; private Filter trrRowFilter; private ResultScanner scanner; - private HTable htable; + private Table htable; private byte [][] trrInputColumns; private long timestamp; private int rowcount; @@ -116,7 +117,7 @@ public class TableRecordReaderImpl { /** * @param htable the {@link HTable} to scan. */ - public void setHTable(HTable htable) { + public void setHTable(Table htable) { Configuration conf = htable.getConfiguration(); logScannerActivity = conf.getBoolean( ScannerCallable.LOG_SCANNER_ACTIVITY, false); diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/mapreduce/DefaultVisibilityExpressionResolver.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/mapreduce/DefaultVisibilityExpressionResolver.java index 62bfba4c365..d09601d96b4 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/mapreduce/DefaultVisibilityExpressionResolver.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/mapreduce/DefaultVisibilityExpressionResolver.java @@ -36,6 +36,7 @@ import org.apache.hadoop.hbase.client.HTable; import org.apache.hadoop.hbase.client.Result; import org.apache.hadoop.hbase.client.ResultScanner; import org.apache.hadoop.hbase.client.Scan; +import org.apache.hadoop.hbase.client.Table; import org.apache.hadoop.hbase.security.visibility.Authorizations; import org.apache.hadoop.hbase.security.visibility.VisibilityLabelOrdinalProvider; import org.apache.hadoop.hbase.security.visibility.VisibilityUtils; @@ -66,7 +67,7 @@ public class DefaultVisibilityExpressionResolver implements VisibilityExpression public void init() { // Reading all the labels and ordinal. // This scan should be done by user with global_admin previliges.. Ensure that it works - HTable labelsTable = null; + Table labelsTable = null; try { labelsTable = new HTable(conf, LABELS_TABLE_NAME); } catch (TableNotFoundException e) { diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/mapreduce/HFileOutputFormat.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/mapreduce/HFileOutputFormat.java index df063a41c1c..24ca01342b8 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/mapreduce/HFileOutputFormat.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/mapreduce/HFileOutputFormat.java @@ -29,6 +29,7 @@ import org.apache.hadoop.classification.InterfaceStability; import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.hbase.KeyValue; import org.apache.hadoop.hbase.client.HTable; +import org.apache.hadoop.hbase.client.Table; import org.apache.hadoop.hbase.io.ImmutableBytesWritable; import org.apache.hadoop.hbase.io.compress.Compression.Algorithm; import org.apache.hadoop.hbase.io.encoding.DataBlockEncoding; @@ -160,7 +161,7 @@ public class HFileOutputFormat extends FileOutputFormat getRegionStartKeys(HTable table) + private static List getRegionStartKeys(RegionLocator table) throws IOException { byte[][] byteKeys = table.getStartKeys(); ArrayList ret = @@ -544,7 +546,7 @@ public class HFileOutputFormat2 value="RCN_REDUNDANT_NULLCHECK_OF_NONNULL_VALUE") @VisibleForTesting static void configureCompression( - HTable table, Configuration conf) throws IOException { + Table table, Configuration conf) throws IOException { StringBuilder compressionConfigValue = new StringBuilder(); HTableDescriptor tableDescriptor = table.getTableDescriptor(); if(tableDescriptor == null){ @@ -578,7 +580,7 @@ public class HFileOutputFormat2 */ @VisibleForTesting static void configureBlockSize( - HTable table, Configuration conf) throws IOException { + Table table, Configuration conf) throws IOException { StringBuilder blockSizeConfigValue = new StringBuilder(); HTableDescriptor tableDescriptor = table.getTableDescriptor(); if (tableDescriptor == null) { @@ -612,7 +614,7 @@ public class HFileOutputFormat2 */ @VisibleForTesting static void configureBloomType( - HTable table, Configuration conf) throws IOException { + Table table, Configuration conf) throws IOException { HTableDescriptor tableDescriptor = table.getTableDescriptor(); if (tableDescriptor == null) { // could happen with mock table instance @@ -647,7 +649,7 @@ public class HFileOutputFormat2 * on failure to read column family descriptors */ @VisibleForTesting - static void configureDataBlockEncoding(HTable table, + static void configureDataBlockEncoding(Table table, Configuration conf) throws IOException { HTableDescriptor tableDescriptor = table.getTableDescriptor(); if (tableDescriptor == null) { diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/mapreduce/HRegionPartitioner.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/mapreduce/HRegionPartitioner.java index 02727cc0c2a..e3e9dd0f7b8 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/mapreduce/HRegionPartitioner.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/mapreduce/HRegionPartitioner.java @@ -28,6 +28,7 @@ import org.apache.hadoop.conf.Configurable; import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.hbase.HBaseConfiguration; import org.apache.hadoop.hbase.client.HTable; +import org.apache.hadoop.hbase.client.RegionLocator; import org.apache.hadoop.hbase.io.ImmutableBytesWritable; import org.apache.hadoop.hbase.util.Bytes; import org.apache.hadoop.mapreduce.Partitioner; @@ -53,7 +54,7 @@ implements Configurable { private static final Log LOG = LogFactory.getLog(HRegionPartitioner.class); private Configuration conf = null; - private HTable table; + private RegionLocator table; private byte[][] startKeys; /** diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/mapreduce/ImportTsv.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/mapreduce/ImportTsv.java index 7bbf71f81d1..be21d64f298 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/mapreduce/ImportTsv.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/mapreduce/ImportTsv.java @@ -39,6 +39,7 @@ import org.apache.hadoop.hbase.HColumnDescriptor; import org.apache.hadoop.hbase.HConstants; import org.apache.hadoop.hbase.HTableDescriptor; import org.apache.hadoop.hbase.TableName; +import org.apache.hadoop.hbase.client.Admin; import org.apache.hadoop.hbase.client.HBaseAdmin; import org.apache.hadoop.hbase.client.HTable; import org.apache.hadoop.hbase.client.Put; @@ -469,7 +470,7 @@ public class ImportTsv extends Configured implements Tool { return job; } - private static void createTable(HBaseAdmin admin, String tableName, String[] columns) + private static void createTable(Admin admin, String tableName, String[] columns) throws IOException { HTableDescriptor htd = new HTableDescriptor(TableName.valueOf(tableName)); Set cfSet = new HashSet(); diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/mapreduce/LoadIncrementalHFiles.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/mapreduce/LoadIncrementalHFiles.java index 3e61d958e21..6fa39aff3cc 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/mapreduce/LoadIncrementalHFiles.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/mapreduce/LoadIncrementalHFiles.java @@ -41,7 +41,6 @@ import java.util.concurrent.Future; import java.util.concurrent.LinkedBlockingQueue; import java.util.concurrent.ThreadPoolExecutor; import java.util.concurrent.TimeUnit; -import java.util.concurrent.atomic.AtomicLong; import org.apache.commons.lang.mutable.MutableInt; import org.apache.commons.logging.Log; @@ -63,11 +62,13 @@ import org.apache.hadoop.hbase.KeyValue; import org.apache.hadoop.hbase.KeyValueUtil; import org.apache.hadoop.hbase.TableName; import org.apache.hadoop.hbase.TableNotFoundException; +import org.apache.hadoop.hbase.client.Admin; import org.apache.hadoop.hbase.client.HBaseAdmin; import org.apache.hadoop.hbase.client.HConnection; import org.apache.hadoop.hbase.client.HTable; import org.apache.hadoop.hbase.client.RegionServerCallable; import org.apache.hadoop.hbase.client.RpcRetryingCallerFactory; +import org.apache.hadoop.hbase.client.Table; import org.apache.hadoop.hbase.client.coprocessor.SecureBulkLoadClient; import org.apache.hadoop.hbase.io.HFileLink; import org.apache.hadoop.hbase.io.HalfStoreFileReader; @@ -106,7 +107,7 @@ import java.util.UUID; @InterfaceStability.Stable public class LoadIncrementalHFiles extends Configured implements Tool { private static final Log LOG = LogFactory.getLog(LoadIncrementalHFiles.class); - private HBaseAdmin hbAdmin; + private Admin hbAdmin; public static final String NAME = "completebulkload"; public static final String MAX_FILES_PER_REGION_PER_FAMILY @@ -359,7 +360,7 @@ public class LoadIncrementalHFiles extends Configured implements Tool { * them. Any failures are re-queued for another pass with the * groupOrSplitPhase. */ - protected void bulkLoadPhase(final HTable table, final HConnection conn, + protected void bulkLoadPhase(final Table table, final HConnection conn, ExecutorService pool, Deque queue, final Multimap regionGroups) throws IOException { // atomically bulk load the groups. @@ -482,7 +483,7 @@ public class LoadIncrementalHFiles extends Configured implements Tool { } protected List splitStoreFile(final LoadQueueItem item, - final HTable table, byte[] startKey, + final Table table, byte[] startKey, byte[] splitKey) throws IOException { final Path hfilePath = item.hfilePath; @@ -646,7 +647,7 @@ public class LoadIncrementalHFiles extends Configured implements Tool { if(!userProvider.isHBaseSecurityEnabled()) { success = ProtobufUtil.bulkLoadHFile(getStub(), famPaths, regionName, assignSeqIds); } else { - HTable table = new HTable(conn.getConfiguration(), getTableName()); + Table table = new HTable(conn.getConfiguration(), getTableName()); secureClient = new SecureBulkLoadClient(table); success = secureClient.bulkLoadHFiles(famPaths, fsDelegationToken.getUserToken(), bulkToken, getLocation().getRegionInfo().getStartKey()); diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/mapreduce/MultiTableInputFormatBase.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/mapreduce/MultiTableInputFormatBase.java index c50e3d55479..15036eed3f6 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/mapreduce/MultiTableInputFormatBase.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/mapreduce/MultiTableInputFormatBase.java @@ -31,6 +31,7 @@ import org.apache.hadoop.hbase.HRegionLocation; import org.apache.hadoop.hbase.client.HTable; import org.apache.hadoop.hbase.client.Result; import org.apache.hadoop.hbase.client.Scan; +import org.apache.hadoop.hbase.client.Table; import org.apache.hadoop.hbase.io.ImmutableBytesWritable; import org.apache.hadoop.hbase.util.Bytes; import org.apache.hadoop.hbase.util.Pair; @@ -84,7 +85,7 @@ public abstract class MultiTableInputFormatBase extends + " previous error. Please look at the previous logs lines from" + " the task's full log for more details."); } - HTable table = + Table table = new HTable(context.getConfiguration(), tSplit.getTableName()); TableRecordReader trr = this.tableRecordReader; diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/mapreduce/TableInputFormatBase.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/mapreduce/TableInputFormatBase.java index e23ea37c6c8..74160930714 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/mapreduce/TableInputFormatBase.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/mapreduce/TableInputFormatBase.java @@ -19,10 +19,8 @@ package org.apache.hadoop.hbase.mapreduce; import java.io.IOException; -import java.io.InterruptedIOException; import java.net.InetAddress; import java.net.InetSocketAddress; -import java.text.MessageFormat; import java.util.ArrayList; import java.util.HashMap; import java.util.List; diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/mapreduce/TableOutputFormat.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/mapreduce/TableOutputFormat.java index 903387183e3..4917045fcb0 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/mapreduce/TableOutputFormat.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/mapreduce/TableOutputFormat.java @@ -32,6 +32,7 @@ import org.apache.hadoop.hbase.client.Delete; import org.apache.hadoop.hbase.client.HTable; import org.apache.hadoop.hbase.client.Mutation; import org.apache.hadoop.hbase.client.Put; +import org.apache.hadoop.hbase.client.Table; import org.apache.hadoop.hbase.zookeeper.ZKUtil; import org.apache.hadoop.mapreduce.JobContext; import org.apache.hadoop.mapreduce.OutputCommitter; @@ -88,14 +89,14 @@ implements Configurable { extends RecordWriter { /** The table to write to. */ - private HTable table; + private Table table; /** * Instantiate a TableRecordWriter with the HBase HClient for writing. * * @param table The table to write to. */ - public TableRecordWriter(HTable table) { + public TableRecordWriter(Table table) { this.table = table; } diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/mapreduce/TableRecordReader.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/mapreduce/TableRecordReader.java index 7db2c560488..825729d9e14 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/mapreduce/TableRecordReader.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/mapreduce/TableRecordReader.java @@ -25,6 +25,7 @@ import org.apache.hadoop.classification.InterfaceStability; import org.apache.hadoop.hbase.client.HTable; import org.apache.hadoop.hbase.client.Result; import org.apache.hadoop.hbase.client.Scan; +import org.apache.hadoop.hbase.client.Table; import org.apache.hadoop.hbase.io.ImmutableBytesWritable; import org.apache.hadoop.mapreduce.InputSplit; import org.apache.hadoop.mapreduce.RecordReader; @@ -57,7 +58,7 @@ extends RecordReader { * * @param htable The {@link HTable} to scan. */ - public void setHTable(HTable htable) { + public void setHTable(Table htable) { this.recordReaderImpl.setHTable(htable); } diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/mapreduce/TableRecordReaderImpl.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/mapreduce/TableRecordReaderImpl.java index e8e6e8bb73e..91b1c1c31a1 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/mapreduce/TableRecordReaderImpl.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/mapreduce/TableRecordReaderImpl.java @@ -31,6 +31,7 @@ import org.apache.hadoop.hbase.client.Result; import org.apache.hadoop.hbase.client.ResultScanner; import org.apache.hadoop.hbase.client.Scan; import org.apache.hadoop.hbase.client.ScannerCallable; +import org.apache.hadoop.hbase.client.Table; import org.apache.hadoop.hbase.client.metrics.ScanMetrics; import org.apache.hadoop.hbase.io.ImmutableBytesWritable; import org.apache.hadoop.hbase.protobuf.ProtobufUtil; @@ -58,7 +59,7 @@ public class TableRecordReaderImpl { private ResultScanner scanner = null; private Scan scan = null; private Scan currentScan = null; - private HTable htable = null; + private Table htable = null; private byte[] lastSuccessfulRow = null; private ImmutableBytesWritable key = null; private Result value = null; @@ -121,7 +122,7 @@ public class TableRecordReaderImpl { * * @param htable The {@link HTable} to scan. */ - public void setHTable(HTable htable) { + public void setHTable(Table htable) { Configuration conf = htable.getConfiguration(); logScannerActivity = conf.getBoolean( ScannerCallable.LOG_SCANNER_ACTIVITY, false); diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/mapreduce/replication/VerifyReplication.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/mapreduce/replication/VerifyReplication.java index 7748675f40d..61caf587952 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/mapreduce/replication/VerifyReplication.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/mapreduce/replication/VerifyReplication.java @@ -33,6 +33,7 @@ import org.apache.hadoop.hbase.client.Put; import org.apache.hadoop.hbase.client.Result; import org.apache.hadoop.hbase.client.ResultScanner; import org.apache.hadoop.hbase.client.Scan; +import org.apache.hadoop.hbase.client.Table; import org.apache.hadoop.hbase.io.ImmutableBytesWritable; import org.apache.hadoop.hbase.mapreduce.TableInputFormat; import org.apache.hadoop.hbase.mapreduce.TableMapReduceUtil; @@ -120,7 +121,7 @@ public class VerifyReplication extends Configured implements Tool { Configuration peerConf = HBaseConfiguration.create(conf); ZKUtil.applyClusterKeyToConf(peerConf, zkClusterKey); - HTable replicatedTable = new HTable(peerConf, conf.get(NAME + ".tableName")); + Table replicatedTable = new HTable(peerConf, conf.get(NAME + ".tableName")); scan.setStartRow(value.getRow()); replicatedScanner = replicatedTable.getScanner(scan); return null; diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/HMasterCommandLine.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/HMasterCommandLine.java index 01a1bf184cb..5a03c432f40 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/HMasterCommandLine.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/HMasterCommandLine.java @@ -30,7 +30,6 @@ import org.apache.commons.logging.Log; import org.apache.commons.logging.LogFactory; import org.apache.hadoop.classification.InterfaceAudience; import org.apache.hadoop.conf.Configuration; -import org.apache.hadoop.fs.FileSystem; import org.apache.hadoop.hbase.CoordinatedStateManager; import org.apache.hadoop.hbase.CoordinatedStateManagerFactory; import org.apache.hadoop.hbase.MasterNotRunningException; @@ -38,6 +37,7 @@ import org.apache.hadoop.hbase.ZNodeClearer; import org.apache.hadoop.hbase.HConstants; import org.apache.hadoop.hbase.LocalHBaseCluster; import org.apache.hadoop.hbase.ZooKeeperConnectionException; +import org.apache.hadoop.hbase.client.Admin; import org.apache.hadoop.hbase.client.HBaseAdmin; import org.apache.hadoop.hbase.regionserver.HRegionServer; import org.apache.hadoop.hbase.util.JVMClusterUtil; @@ -209,7 +209,7 @@ public class HMasterCommandLine extends ServerCommandLine { } private int stopMaster() { - HBaseAdmin adm = null; + Admin adm = null; try { Configuration conf = getConf(); // Don't try more than once diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/RegionPlacementMaintainer.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/RegionPlacementMaintainer.java index 282641e503e..ef205be64be 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/RegionPlacementMaintainer.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/RegionPlacementMaintainer.java @@ -47,6 +47,7 @@ import org.apache.hadoop.hbase.HConstants; import org.apache.hadoop.hbase.HRegionInfo; import org.apache.hadoop.hbase.ServerName; import org.apache.hadoop.hbase.TableName; +import org.apache.hadoop.hbase.client.Admin; import org.apache.hadoop.hbase.client.HBaseAdmin; import org.apache.hadoop.hbase.client.HConnection; import org.apache.hadoop.hbase.client.HConnectionManager; @@ -92,7 +93,7 @@ public class RegionPlacementMaintainer { private Configuration conf; private final boolean enforceLocality; private final boolean enforceMinAssignmentMove; - private HBaseAdmin admin; + private Admin admin; private RackManager rackManager; private Set targetTableSet; @@ -127,7 +128,7 @@ public class RegionPlacementMaintainer { * @return the cached HBaseAdmin * @throws IOException */ - private HBaseAdmin getHBaseAdmin() throws IOException { + private Admin getHBaseAdmin() throws IOException { if (this.admin == null) { this.admin = new HBaseAdmin(this.conf); } diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/TableNamespaceManager.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/TableNamespaceManager.java index 6a83dc3dbb5..a78e225c586 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/TableNamespaceManager.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/TableNamespaceManager.java @@ -45,6 +45,7 @@ import org.apache.hadoop.hbase.client.HTable; import org.apache.hadoop.hbase.client.Put; import org.apache.hadoop.hbase.client.Result; import org.apache.hadoop.hbase.client.ResultScanner; +import org.apache.hadoop.hbase.client.Table; import org.apache.hadoop.hbase.constraint.ConstraintException; import org.apache.hadoop.hbase.master.handler.CreateTableHandler; import org.apache.hadoop.hbase.protobuf.ProtobufUtil; @@ -107,7 +108,7 @@ public class TableNamespaceManager { isTableAvailableAndInitialized(); } - private synchronized HTable getNamespaceTable() throws IOException { + private synchronized Table getNamespaceTable() throws IOException { if (!isTableAvailableAndInitialized()) { throw new IOException(this.getClass().getName() + " isn't ready to serve"); } @@ -125,14 +126,14 @@ public class TableNamespaceManager { } public synchronized void update(NamespaceDescriptor ns) throws IOException { - HTable table = getNamespaceTable(); + Table table = getNamespaceTable(); if (get(table, ns.getName()) == null) { throw new NamespaceNotFoundException(ns.getName()); } upsert(table, ns); } - private NamespaceDescriptor get(HTable table, String name) throws IOException { + private NamespaceDescriptor get(Table table, String name) throws IOException { Result res = table.get(new Get(Bytes.toBytes(name))); if (res.isEmpty()) { return null; @@ -144,7 +145,7 @@ public class TableNamespaceManager { HBaseProtos.NamespaceDescriptor.parseFrom(val)); } - private void create(HTable table, NamespaceDescriptor ns) throws IOException { + private void create(Table table, NamespaceDescriptor ns) throws IOException { if (get(table, ns.getName()) != null) { throw new NamespaceExistException(ns.getName()); } @@ -154,7 +155,7 @@ public class TableNamespaceManager { upsert(table, ns); } - private void upsert(HTable table, NamespaceDescriptor ns) throws IOException { + private void upsert(Table table, NamespaceDescriptor ns) throws IOException { Put p = new Put(Bytes.toBytes(ns.getName())); p.addImmutable(HTableDescriptor.NAMESPACE_FAMILY_INFO_BYTES, HTableDescriptor.NAMESPACE_COL_DESC_BYTES, diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/balancer/FavoredNodeAssignmentHelper.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/balancer/FavoredNodeAssignmentHelper.java index 79fd21ec59d..eb6878be305 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/balancer/FavoredNodeAssignmentHelper.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/balancer/FavoredNodeAssignmentHelper.java @@ -41,6 +41,7 @@ import org.apache.hadoop.hbase.MetaTableAccessor; import org.apache.hadoop.hbase.client.HConnection; import org.apache.hadoop.hbase.client.HTable; import org.apache.hadoop.hbase.client.Put; +import org.apache.hadoop.hbase.client.Table; import org.apache.hadoop.hbase.master.RackManager; import org.apache.hadoop.hbase.protobuf.ProtobufUtil; import org.apache.hadoop.hbase.protobuf.generated.HBaseProtos; @@ -120,7 +121,7 @@ public class FavoredNodeAssignmentHelper { } } // Write the region assignments to the meta table. - HTable metaTable = null; + Table metaTable = null; try { metaTable = new HTable(conf, TableName.META_TABLE_NAME); metaTable.put(puts); diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/handler/ModifyTableHandler.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/handler/ModifyTableHandler.java index 48fb26f71d2..591a1d883c9 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/handler/ModifyTableHandler.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/handler/ModifyTableHandler.java @@ -36,6 +36,7 @@ import org.apache.hadoop.hbase.client.HTable; import org.apache.hadoop.hbase.client.Result; import org.apache.hadoop.hbase.client.ResultScanner; import org.apache.hadoop.hbase.client.Scan; +import org.apache.hadoop.hbase.client.Table; import org.apache.hadoop.hbase.executor.EventType; import org.apache.hadoop.hbase.master.HMaster; import org.apache.hadoop.hbase.master.MasterCoprocessorHost; @@ -94,7 +95,7 @@ public class ModifyTableHandler extends TableEventHandler { Set tableRows = new HashSet(); Scan scan = MetaTableAccessor.getScanForTableName(table); scan.addColumn(HConstants.CATALOG_FAMILY, HConstants.REGIONINFO_QUALIFIER); - HTable htable = null; + Table htable = null; try { htable = new HTable(masterServices.getConfiguration(), TableName.META_TABLE_NAME); ResultScanner resScanner = htable.getScanner(scan); diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/ReplicationSink.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/ReplicationSink.java index 4a4de8693c1..d10dc489385 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/ReplicationSink.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/ReplicationSink.java @@ -46,10 +46,10 @@ import org.apache.hadoop.hbase.Stoppable; import org.apache.hadoop.hbase.client.Delete; import org.apache.hadoop.hbase.client.HConnection; import org.apache.hadoop.hbase.client.HConnectionManager; -import org.apache.hadoop.hbase.client.HTableInterface; import org.apache.hadoop.hbase.client.Mutation; import org.apache.hadoop.hbase.client.Put; import org.apache.hadoop.hbase.client.Row; +import org.apache.hadoop.hbase.client.Table; import org.apache.hadoop.hbase.protobuf.generated.AdminProtos.WALEntry; import org.apache.hadoop.hbase.protobuf.generated.HBaseProtos; @@ -230,7 +230,7 @@ public class ReplicationSink { if (allRows.isEmpty()) { return; } - HTableInterface table = null; + Table table = null; try { table = this.sharedHtableCon.getTable(tableName); for (List rows : allRows) { diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/rest/RESTServlet.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/rest/RESTServlet.java index 9295e9dd116..5e7038d90e8 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/rest/RESTServlet.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/rest/RESTServlet.java @@ -23,7 +23,7 @@ import java.io.IOException; import org.apache.hadoop.classification.InterfaceAudience; import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.hbase.client.HBaseAdmin; -import org.apache.hadoop.hbase.client.HTableInterface; +import org.apache.hadoop.hbase.client.Table; import org.apache.hadoop.hbase.filter.ParseFilter; import org.apache.hadoop.hbase.security.UserProvider; import org.apache.hadoop.hbase.util.ConnectionCache; @@ -100,7 +100,7 @@ public class RESTServlet implements Constants { /** * Caller closes the table afterwards. */ - HTableInterface getTable(String tableName) throws IOException { + Table getTable(String tableName) throws IOException { return connectionCache.getTable(tableName); } diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/rest/RowResource.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/rest/RowResource.java index 6f0a9eb7402..84564edca8e 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/rest/RowResource.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/rest/RowResource.java @@ -45,8 +45,8 @@ import org.apache.hadoop.hbase.CellUtil; import org.apache.hadoop.hbase.HConstants; import org.apache.hadoop.hbase.KeyValue; import org.apache.hadoop.hbase.client.Delete; -import org.apache.hadoop.hbase.client.HTableInterface; import org.apache.hadoop.hbase.client.Put; +import org.apache.hadoop.hbase.client.Table; import org.apache.hadoop.hbase.rest.model.CellModel; import org.apache.hadoop.hbase.rest.model.CellSetModel; import org.apache.hadoop.hbase.rest.model.RowModel; @@ -182,7 +182,7 @@ public class RowResource extends ResourceBase { .build(); } - HTableInterface table = null; + Table table = null; try { List rows = model.getRows(); List puts = new ArrayList(); @@ -253,7 +253,7 @@ public class RowResource extends ResourceBase { .type(MIMETYPE_TEXT).entity("Forbidden" + CRLF) .build(); } - HTableInterface table = null; + Table table = null; try { byte[] row = rowspec.getRow(); byte[][] columns = rowspec.getColumns(); @@ -393,7 +393,7 @@ public class RowResource extends ResourceBase { } } } - HTableInterface table = null; + Table table = null; try { table = servlet.getTable(tableResource.getName()); table.delete(delete); @@ -422,7 +422,7 @@ public class RowResource extends ResourceBase { * @return Response 200 OK, 304 Not modified, 400 Bad request */ Response checkAndPut(final CellSetModel model) { - HTableInterface table = null; + Table table = null; try { table = servlet.getTable(tableResource.getName()); if (model.getRows().size() != 1) { @@ -513,7 +513,7 @@ public class RowResource extends ResourceBase { * @return Response 200 OK, 304 Not modified, 400 Bad request */ Response checkAndDelete(final CellSetModel model) { - HTableInterface table = null; + Table table = null; Delete delete = null; try { table = servlet.getTable(tableResource.getName()); diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/rest/RowResultGenerator.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/rest/RowResultGenerator.java index cdfc0417a2b..0b37d550f96 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/rest/RowResultGenerator.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/rest/RowResultGenerator.java @@ -30,8 +30,8 @@ import org.apache.hadoop.hbase.Cell; import org.apache.hadoop.hbase.DoNotRetryIOException; import org.apache.hadoop.hbase.KeyValue; import org.apache.hadoop.hbase.client.Get; -import org.apache.hadoop.hbase.client.HTableInterface; import org.apache.hadoop.hbase.client.Result; +import org.apache.hadoop.hbase.client.Table; import org.apache.hadoop.hbase.filter.Filter; import org.apache.hadoop.util.StringUtils; @@ -45,7 +45,7 @@ public class RowResultGenerator extends ResultGenerator { public RowResultGenerator(final String tableName, final RowSpec rowspec, final Filter filter, final boolean cacheBlocks) throws IllegalArgumentException, IOException { - HTableInterface table = RESTServlet.getInstance().getTable(tableName); + Table table = RESTServlet.getInstance().getTable(tableName); try { Get get = new Get(rowspec.getRow()); if (rowspec.hasColumns()) { diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/rest/ScannerResultGenerator.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/rest/ScannerResultGenerator.java index 32f145bf2fa..261feae21e4 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/rest/ScannerResultGenerator.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/rest/ScannerResultGenerator.java @@ -28,10 +28,10 @@ import org.apache.hadoop.classification.InterfaceAudience; import org.apache.hadoop.hbase.Cell; import org.apache.hadoop.hbase.KeyValue; import org.apache.hadoop.hbase.UnknownScannerException; -import org.apache.hadoop.hbase.client.HTableInterface; import org.apache.hadoop.hbase.client.Result; import org.apache.hadoop.hbase.client.ResultScanner; import org.apache.hadoop.hbase.client.Scan; +import org.apache.hadoop.hbase.client.Table; import org.apache.hadoop.hbase.filter.Filter; import org.apache.hadoop.hbase.rest.model.ScannerModel; import org.apache.hadoop.hbase.security.visibility.Authorizations; @@ -67,7 +67,7 @@ public class ScannerResultGenerator extends ResultGenerator { public ScannerResultGenerator(final String tableName, final RowSpec rowspec, final Filter filter, final int caching, final boolean cacheBlocks) throws IllegalArgumentException, IOException { - HTableInterface table = RESTServlet.getInstance().getTable(tableName); + Table table = RESTServlet.getInstance().getTable(tableName); try { Scan scan; if (rowspec.hasEndRow()) { diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/rest/SchemaResource.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/rest/SchemaResource.java index ae036510800..f85ddacd12c 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/rest/SchemaResource.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/rest/SchemaResource.java @@ -44,7 +44,7 @@ import org.apache.hadoop.hbase.TableExistsException; import org.apache.hadoop.hbase.TableName; import org.apache.hadoop.hbase.TableNotFoundException; import org.apache.hadoop.hbase.client.HBaseAdmin; -import org.apache.hadoop.hbase.client.HTableInterface; +import org.apache.hadoop.hbase.client.Table; import org.apache.hadoop.hbase.rest.model.ColumnSchemaModel; import org.apache.hadoop.hbase.rest.model.TableSchemaModel; import org.apache.hadoop.hbase.util.Bytes; @@ -74,7 +74,7 @@ public class SchemaResource extends ResourceBase { private HTableDescriptor getTableSchema() throws IOException, TableNotFoundException { - HTableInterface table = servlet.getTable(tableResource.getName()); + Table table = servlet.getTable(tableResource.getName()); try { return table.getTableDescriptor(); } finally { diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/rest/TableResource.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/rest/TableResource.java index 0627ed24e3c..5eef017f8e8 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/rest/TableResource.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/rest/TableResource.java @@ -35,8 +35,8 @@ import org.apache.commons.lang.StringUtils; import org.apache.commons.logging.Log; import org.apache.commons.logging.LogFactory; import org.apache.hadoop.classification.InterfaceAudience; -import org.apache.hadoop.hbase.client.HTableInterface; import org.apache.hadoop.hbase.client.Scan; +import org.apache.hadoop.hbase.client.Table; import org.apache.hadoop.hbase.filter.Filter; import org.apache.hadoop.hbase.filter.FilterList; import org.apache.hadoop.hbase.filter.ParseFilter; @@ -144,7 +144,7 @@ public class TableResource extends ResourceBase { + " End Row => " + endRow + " Columns => " + column + " Start Time => " + startTime + " End Time => " + endTime + " Cache Blocks => " + cacheBlocks + " Max Versions => " + maxVersions + " Batch Size => " + batchSize); - HTableInterface hTable = RESTServlet.getInstance().getTable(this.table); + Table hTable = RESTServlet.getInstance().getTable(this.table); Scan tableScan = new Scan(); tableScan.setBatch(batchSize); tableScan.setMaxVersions(maxVersions); diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/security/access/AccessControlLists.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/security/access/AccessControlLists.java index 19e51365f75..16c2c3ae278 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/security/access/AccessControlLists.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/security/access/AccessControlLists.java @@ -51,6 +51,7 @@ import org.apache.hadoop.hbase.client.Put; import org.apache.hadoop.hbase.client.Result; import org.apache.hadoop.hbase.client.ResultScanner; import org.apache.hadoop.hbase.client.Scan; +import org.apache.hadoop.hbase.client.Table; import org.apache.hadoop.hbase.exceptions.DeserializationException; import org.apache.hadoop.hbase.filter.CompareFilter.CompareOp; import org.apache.hadoop.hbase.filter.QualifierFilter; @@ -172,7 +173,7 @@ public class AccessControlLists { Bytes.toString(key)+": "+Bytes.toStringBinary(value) ); } - HTable acls = null; + Table acls = null; try { acls = new HTable(conf, ACL_TABLE_NAME); acls.put(p); @@ -203,7 +204,7 @@ public class AccessControlLists { LOG.debug("Removing permission "+ userPerm.toString()); } d.deleteColumns(ACL_LIST_FAMILY, key); - HTable acls = null; + Table acls = null; try { acls = new HTable(conf, ACL_TABLE_NAME); acls.delete(d); @@ -223,7 +224,7 @@ public class AccessControlLists { LOG.debug("Removing permissions of removed table "+ tableName); } - HTable acls = null; + Table acls = null; try { acls = new HTable(conf, ACL_TABLE_NAME); acls.delete(d); @@ -243,7 +244,7 @@ public class AccessControlLists { LOG.debug("Removing permissions of removed namespace "+ namespace); } - HTable acls = null; + Table acls = null; try { acls = new HTable(conf, ACL_TABLE_NAME); acls.delete(d); @@ -263,7 +264,7 @@ public class AccessControlLists { " from table "+ tableName); } - HTable acls = null; + Table acls = null; try { acls = new HTable(conf, ACL_TABLE_NAME); @@ -424,7 +425,7 @@ public class AccessControlLists { Scan scan = new Scan(); scan.addFamily(ACL_LIST_FAMILY); - HTable acls = null; + Table acls = null; ResultScanner scanner = null; try { acls = new HTable(conf, ACL_TABLE_NAME); @@ -467,7 +468,7 @@ public class AccessControlLists { // for normal user tables, we just read the table row from _acl_ ListMultimap perms = ArrayListMultimap.create(); - HTable acls = null; + Table acls = null; try { acls = new HTable(conf, ACL_TABLE_NAME); Get get = new Get(entryName); diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/security/token/TokenUtil.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/security/token/TokenUtil.java index cdc824dd7ed..78d2c14f647 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/security/token/TokenUtil.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/security/token/TokenUtil.java @@ -30,6 +30,7 @@ import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.hbase.HConstants; import org.apache.hadoop.hbase.TableName; import org.apache.hadoop.hbase.client.HTable; +import org.apache.hadoop.hbase.client.Table; import org.apache.hadoop.hbase.ipc.CoprocessorRpcChannel; import org.apache.hadoop.hbase.protobuf.ProtobufUtil; import org.apache.hadoop.hbase.protobuf.generated.AuthenticationProtos; @@ -53,7 +54,7 @@ public class TokenUtil { */ public static Token obtainToken( Configuration conf) throws IOException { - HTable meta = null; + Table meta = null; try { meta = new HTable(conf, TableName.META_TABLE_NAME); CoprocessorRpcChannel rpcChannel = meta.coprocessorService(HConstants.EMPTY_START_ROW); diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/tool/Canary.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/tool/Canary.java index 489aaf1dd3d..9b11291549f 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/tool/Canary.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/tool/Canary.java @@ -49,6 +49,7 @@ import org.apache.hadoop.hbase.client.HBaseAdmin; import org.apache.hadoop.hbase.client.HTable; import org.apache.hadoop.hbase.client.ResultScanner; import org.apache.hadoop.hbase.client.Scan; +import org.apache.hadoop.hbase.client.Table; import org.apache.hadoop.util.Tool; import org.apache.hadoop.util.ToolRunner; @@ -326,7 +327,7 @@ public final class Canary implements Tool { public static abstract class Monitor implements Runnable { protected Configuration config; - protected HBaseAdmin admin; + protected Admin admin; protected String[] targets; protected boolean useRegExp; protected boolean initialized = false; @@ -476,7 +477,7 @@ public final class Canary implements Tool { */ private static void sniff(final Admin admin, final Sink sink, HTableDescriptor tableDesc) throws Exception { - HTable table = null; + Table table = null; try { table = new HTable(admin.getConfiguration(), tableDesc.getName()); @@ -506,7 +507,7 @@ public final class Canary implements Tool { final Admin admin, final Sink sink, HRegionInfo region, - HTable table) throws Exception { + Table table) throws Exception { HTableDescriptor tableDesc = table.getTableDescriptor(); byte[] startKey = null; Get get = null; @@ -607,7 +608,7 @@ public final class Canary implements Tool { String serverName = null; String tableName = null; HRegionInfo region = null; - HTable table = null; + Table table = null; Get get = null; byte[] startKey = null; Scan scan = null; diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/util/HBaseFsck.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/util/HBaseFsck.java index 982417eb85e..68ba938e819 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/util/HBaseFsck.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/util/HBaseFsck.java @@ -72,6 +72,7 @@ import org.apache.hadoop.hbase.ServerName; import org.apache.hadoop.hbase.TableName; import org.apache.hadoop.hbase.ZooKeeperConnectionException; import org.apache.hadoop.hbase.MetaTableAccessor; +import org.apache.hadoop.hbase.client.Admin; import org.apache.hadoop.hbase.client.Delete; import org.apache.hadoop.hbase.client.Get; import org.apache.hadoop.hbase.client.HBaseAdmin; @@ -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.Result; import org.apache.hadoop.hbase.client.RowMutations; +import org.apache.hadoop.hbase.client.Table; import org.apache.hadoop.hbase.io.hfile.CacheConfig; import org.apache.hadoop.hbase.io.hfile.HFile; import org.apache.hadoop.hbase.master.MasterFileSystem; @@ -183,8 +185,8 @@ public class HBaseFsck extends Configured { private static final Log LOG = LogFactory.getLog(HBaseFsck.class.getName()); private ClusterStatus status; private HConnection connection; - private HBaseAdmin admin; - private HTable meta; + private Admin admin; + private Table meta; // threads to do ||izable tasks: retrieve data from regionservers, handle overlapping regions protected ExecutorService executor; private long startMillis = System.currentTimeMillis(); @@ -2722,7 +2724,7 @@ public class HBaseFsck extends Configured { HTableDescriptor[] getHTableDescriptors(List tableNames) { HTableDescriptor[] htd = new HTableDescriptor[0]; - HBaseAdmin admin = null; + Admin admin = null; try { LOG.info("getHTableDescriptors == tableNames => " + tableNames); admin = new HBaseAdmin(getConf()); diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/util/HBaseFsckRepair.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/util/HBaseFsckRepair.java index 641fd36e979..ab73f6bb4e4 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/util/HBaseFsckRepair.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/util/HBaseFsckRepair.java @@ -34,9 +34,10 @@ import org.apache.hadoop.hbase.TableName; import org.apache.hadoop.hbase.ZooKeeperConnectionException; import org.apache.hadoop.hbase.MetaTableAccessor; import org.apache.hadoop.hbase.client.Admin; -import org.apache.hadoop.hbase.client.HBaseAdmin; import org.apache.hadoop.hbase.client.HConnection; import org.apache.hadoop.hbase.client.HTable; +import org.apache.hadoop.hbase.client.Put; +import org.apache.hadoop.hbase.client.Table; import org.apache.hadoop.hbase.master.RegionState; import org.apache.hadoop.hbase.protobuf.ProtobufUtil; import org.apache.hadoop.hbase.protobuf.generated.AdminProtos.AdminService; @@ -60,7 +61,7 @@ public class HBaseFsckRepair { * @param region Region to undeploy * @param servers list of Servers to undeploy from */ - public static void fixMultiAssignment(HBaseAdmin admin, HRegionInfo region, + public static void fixMultiAssignment(Admin admin, HRegionInfo region, List servers) throws IOException, KeeperException, InterruptedException { HRegionInfo actualRegion = new HRegionInfo(region); @@ -86,7 +87,7 @@ public class HBaseFsckRepair { * @throws IOException * @throws KeeperException */ - public static void fixUnassigned(HBaseAdmin admin, HRegionInfo region) + public static void fixUnassigned(Admin admin, HRegionInfo region) throws IOException, KeeperException { HRegionInfo actualRegion = new HRegionInfo(region); @@ -106,7 +107,7 @@ public class HBaseFsckRepair { * side-effect of requiring a HRegionInfo that considers regionId (timestamp) * in comparators that is addressed by HBASE-5563. */ - private static void forceOfflineInZK(HBaseAdmin admin, final HRegionInfo region) + private static void forceOfflineInZK(Admin admin, final HRegionInfo region) throws ZooKeeperConnectionException, KeeperException, IOException { admin.assign(region.getRegionName()); } @@ -114,7 +115,7 @@ public class HBaseFsckRepair { /* * Should we check all assignments or just not in RIT? */ - public static void waitUntilAssigned(HBaseAdmin admin, + public static void waitUntilAssigned(Admin admin, HRegionInfo region) throws IOException, InterruptedException { long timeout = admin.getConfiguration().getLong("hbase.hbck.assign.timeout", 120000); long expiration = timeout + System.currentTimeMillis(); @@ -175,7 +176,7 @@ public class HBaseFsckRepair { */ public static void fixMetaHoleOnline(Configuration conf, 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); meta.close(); } diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/util/HMerge.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/util/HMerge.java index 0f2c5db9888..50f0c0a4b2a 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/util/HMerge.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/util/HMerge.java @@ -37,6 +37,7 @@ import org.apache.hadoop.hbase.HTableDescriptor; import org.apache.hadoop.hbase.RemoteExceptionHandler; import org.apache.hadoop.hbase.TableNotDisabledException; 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.HBaseAdmin; 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.Result; import org.apache.hadoop.hbase.client.ResultScanner; +import org.apache.hadoop.hbase.client.Table; import org.apache.hadoop.hbase.regionserver.HRegion; import org.apache.hadoop.hbase.regionserver.wal.HLog; import org.apache.hadoop.hbase.regionserver.wal.HLogFactory; @@ -124,7 +126,7 @@ class HMerge { throw new IllegalStateException( "HBase instance must be running to merge a normal table"); } - HBaseAdmin admin = new HBaseAdmin(conf); + Admin admin = new HBaseAdmin(conf); try { if (!admin.isTableDisabled(tableName)) { throw new TableNotDisabledException(tableName); @@ -231,7 +233,7 @@ class HMerge { /** Instantiated to compact a normal user table */ private static class OnlineMerger extends Merger { private final TableName tableName; - private final HTable table; + private final Table table; private final ResultScanner metaScanner; private HRegionInfo latestRegion; diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/util/RegionSizeCalculator.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/util/RegionSizeCalculator.java index fa5500b20aa..9284f039b05 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/util/RegionSizeCalculator.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/util/RegionSizeCalculator.java @@ -27,11 +27,11 @@ import org.apache.hadoop.hbase.HRegionInfo; import org.apache.hadoop.hbase.RegionLoad; import org.apache.hadoop.hbase.ServerLoad; import org.apache.hadoop.hbase.ServerName; +import org.apache.hadoop.hbase.client.Admin; import org.apache.hadoop.hbase.client.HBaseAdmin; import org.apache.hadoop.hbase.client.HTable; import java.io.IOException; -import java.text.MessageFormat; import java.util.Arrays; import java.util.Collection; import java.util.Collections; @@ -65,7 +65,7 @@ public class RegionSizeCalculator { } /** ctor for unit testing */ - RegionSizeCalculator (HTable table, HBaseAdmin admin) throws IOException { + RegionSizeCalculator (HTable table, Admin admin) throws IOException { try { if (!enabled(table.getConfiguration())) { diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/HBaseTestCase.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/HBaseTestCase.java index e08530d2c3a..db89a459a19 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/HBaseTestCase.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/HBaseTestCase.java @@ -39,6 +39,7 @@ import org.apache.hadoop.hbase.client.Put; import org.apache.hadoop.hbase.client.Result; import org.apache.hadoop.hbase.client.ResultScanner; import org.apache.hadoop.hbase.client.Scan; +import org.apache.hadoop.hbase.client.Table; import org.apache.hadoop.hbase.regionserver.HRegion; import org.apache.hadoop.hbase.regionserver.InternalScanner; import org.apache.hadoop.hbase.util.Bytes; @@ -467,12 +468,12 @@ public abstract class HBaseTestCase extends TestCase { * A class that makes a {@link Incommon} out of a {@link HTable} */ public static class HTableIncommon implements Incommon { - final HTable table; + final Table table; /** * @param table */ - public HTableIncommon(final HTable table) { + public HTableIncommon(final Table table) { super(); this.table = table; } diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/HBaseTestingUtility.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/HBaseTestingUtility.java index c67f3f81792..cf60b554d24 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/HBaseTestingUtility.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/HBaseTestingUtility.java @@ -63,9 +63,11 @@ import org.apache.hadoop.hbase.client.HConnection; import org.apache.hadoop.hbase.client.HConnectionManager; import org.apache.hadoop.hbase.client.HTable; import org.apache.hadoop.hbase.client.Put; +import org.apache.hadoop.hbase.client.RegionLocator; import org.apache.hadoop.hbase.client.Result; import org.apache.hadoop.hbase.client.ResultScanner; import org.apache.hadoop.hbase.client.Scan; +import org.apache.hadoop.hbase.client.Table; import org.apache.hadoop.hbase.fs.HFileSystem; import org.apache.hadoop.hbase.io.compress.Compression; import org.apache.hadoop.hbase.io.compress.Compression.Algorithm; @@ -899,7 +901,7 @@ public class HBaseTestingUtility extends HBaseCommonTestingUtility { this.hbaseCluster = new MiniHBaseCluster(c, numMasters, numSlaves, masterClass, regionserverClass); // Don't leave here till we've done a successful scan of the hbase:meta - HTable t = new HTable(c, TableName.META_TABLE_NAME); + Table t = new HTable(c, TableName.META_TABLE_NAME); ResultScanner s = t.getScanner(new Scan()); while (s.next() != null) { continue; @@ -921,7 +923,7 @@ public class HBaseTestingUtility extends HBaseCommonTestingUtility { public void restartHBaseCluster(int servers) throws IOException, InterruptedException { this.hbaseCluster = new MiniHBaseCluster(this.conf, servers); // Don't leave here till we've done a successful scan of the hbase:meta - HTable t = new HTable(new Configuration(this.conf), TableName.META_TABLE_NAME); + Table t = new HTable(new Configuration(this.conf), TableName.META_TABLE_NAME); ResultScanner s = t.getScanner(new Scan()); while (s.next() != null) { // do nothing @@ -1848,7 +1850,7 @@ public class HBaseTestingUtility extends HBaseCommonTestingUtility { 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++) { byte[] data = Bytes.toBytes(String.valueOf(i)); 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++) { byte[] data = Bytes.toBytes(String.valueOf(i)); Delete delete = new Delete(data); @@ -1869,7 +1871,7 @@ public class HBaseTestingUtility extends HBaseCommonTestingUtility { /** * Return the number of rows in the given table. */ - public int countRows(final HTable table) throws IOException { + public int countRows(final Table table) throws IOException { Scan scan = new Scan(); ResultScanner results = table.getScanner(scan); int count = 0; @@ -1880,7 +1882,7 @@ public class HBaseTestingUtility extends HBaseCommonTestingUtility { return count; } - public int countRows(final HTable table, final byte[]... families) throws IOException { + public int countRows(final Table table, final byte[]... families) throws IOException { Scan scan = new Scan(); for (byte[] family: families) { scan.addFamily(family); @@ -1897,7 +1899,7 @@ public class HBaseTestingUtility extends HBaseCommonTestingUtility { /** * Return an md5 digest of the entire contents of a table. */ - public String checksumRows(final HTable table) throws Exception { + public String checksumRows(final Table table) throws Exception { Scan scan = new Scan(); ResultScanner results = table.getScanner(scan); MessageDigest digest = MessageDigest.getInstance("MD5"); @@ -2003,7 +2005,7 @@ public class HBaseTestingUtility extends HBaseCommonTestingUtility { final byte[] columnFamily, byte [][] startKeys) throws IOException { Arrays.sort(startKeys, Bytes.BYTES_COMPARATOR); - HTable meta = new HTable(c, TableName.META_TABLE_NAME); + Table meta = new HTable(c, TableName.META_TABLE_NAME); HTableDescriptor htd = table.getTableDescriptor(); if(!htd.hasFamily(columnFamily)) { HColumnDescriptor hcd = new HColumnDescriptor(columnFamily); @@ -2068,7 +2070,7 @@ public class HBaseTestingUtility extends HBaseCommonTestingUtility { public List createMultiRegionsInMeta(final Configuration conf, final HTableDescriptor htd, byte [][] startKeys) throws IOException { - HTable meta = new HTable(conf, TableName.META_TABLE_NAME); + Table meta = new HTable(conf, TableName.META_TABLE_NAME); Arrays.sort(startKeys, Bytes.BYTES_COMPARATOR); List newRegions = new ArrayList(startKeys.length); // add custom ones @@ -2091,7 +2093,7 @@ public class HBaseTestingUtility extends HBaseCommonTestingUtility { */ public List getMetaTableRows() throws IOException { // TODO: Redo using MetaTableAccessor class - HTable t = new HTable(new Configuration(this.conf), TableName.META_TABLE_NAME); + Table t = new HTable(new Configuration(this.conf), TableName.META_TABLE_NAME); List rows = new ArrayList(); ResultScanner s = t.getScanner(new Scan()); for (Result result : s) { @@ -2111,7 +2113,7 @@ public class HBaseTestingUtility extends HBaseCommonTestingUtility { */ public List getMetaTableRows(TableName tableName) throws IOException { // TODO: Redo using MetaTableAccessor. - HTable t = new HTable(new Configuration(this.conf), TableName.META_TABLE_NAME); + Table t = new HTable(new Configuration(this.conf), TableName.META_TABLE_NAME); List rows = new ArrayList(); ResultScanner s = t.getScanner(new Scan()); for (Result result : s) { @@ -2557,7 +2559,7 @@ public class HBaseTestingUtility extends HBaseCommonTestingUtility { * @param table The table to find the region. * @throws IOException */ - public void closeRegionByRow(String row, HTable table) throws IOException { + public void closeRegionByRow(String row, RegionLocator table) throws IOException { closeRegionByRow(Bytes.toBytes(row), table); } @@ -2568,7 +2570,7 @@ public class HBaseTestingUtility extends HBaseCommonTestingUtility { * @param table The table to find the region. * @throws IOException */ - public void closeRegionByRow(byte[] row, HTable table) throws IOException { + public void closeRegionByRow(byte[] row, RegionLocator table) throws IOException { HRegionLocation hrl = table.getRegionLocation(row); closeRegion(hrl.getRegionInfo().getRegionName()); } @@ -2894,7 +2896,7 @@ public class HBaseTestingUtility extends HBaseCommonTestingUtility { */ public void waitUntilAllRegionsAssigned(final TableName tableName, final long timeout) throws IOException { - final HTable meta = new HTable(getConfiguration(), TableName.META_TABLE_NAME); + final Table meta = new HTable(getConfiguration(), TableName.META_TABLE_NAME); try { waitFor(timeout, 200, true, new Predicate() { @Override @@ -3314,7 +3316,7 @@ public class HBaseTestingUtility extends HBaseCommonTestingUtility { } public static int getMetaRSPort(Configuration conf) throws IOException { - HTable table = new HTable(conf, TableName.META_TABLE_NAME); + RegionLocator table = new HTable(conf, TableName.META_TABLE_NAME); HRegionLocation hloc = table.getRegionLocation(Bytes.toBytes("")); table.close(); return hloc.getPort(); diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/PerformanceEvaluation.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/PerformanceEvaluation.java index 5390bb140db..0c47a9ef0d0 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/PerformanceEvaluation.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/PerformanceEvaluation.java @@ -58,11 +58,11 @@ import org.apache.hadoop.hbase.client.HBaseAdmin; import org.apache.hadoop.hbase.client.HConnection; import org.apache.hadoop.hbase.client.HConnectionManager; import org.apache.hadoop.hbase.client.HTable; -import org.apache.hadoop.hbase.client.HTableInterface; import org.apache.hadoop.hbase.client.Put; import org.apache.hadoop.hbase.client.Result; import org.apache.hadoop.hbase.client.ResultScanner; import org.apache.hadoop.hbase.client.Scan; +import org.apache.hadoop.hbase.client.Table; import org.apache.hadoop.hbase.filter.BinaryComparator; import org.apache.hadoop.hbase.filter.CompareFilter; import org.apache.hadoop.hbase.filter.Filter; @@ -892,7 +892,7 @@ public class PerformanceEvaluation extends Configured implements Tool { private final Sampler traceSampler; private final SpanReceiverHost receiverHost; protected HConnection connection; - protected HTableInterface table; + protected Table table; private String testName; private Histogram latency; @@ -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 // the TestOptions introspection for us and dump the output in a readable format. LOG.info(cmd.getSimpleName() + " test run options=" + MAPPER.writeValueAsString(opts)); - HBaseAdmin admin = null; + Admin admin = null; try { admin = new HBaseAdmin(getConf()); checkTable(admin, opts); diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/ScanPerformanceEvaluation.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/ScanPerformanceEvaluation.java index 484a00c6990..dfafa836c85 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/ScanPerformanceEvaluation.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/ScanPerformanceEvaluation.java @@ -30,6 +30,7 @@ import org.apache.hadoop.hbase.client.HTable; import org.apache.hadoop.hbase.client.Result; import org.apache.hadoop.hbase.client.ResultScanner; import org.apache.hadoop.hbase.client.Scan; +import org.apache.hadoop.hbase.client.Table; import org.apache.hadoop.hbase.client.TableSnapshotScanner; import org.apache.hadoop.hbase.client.metrics.ScanMetrics; import org.apache.hadoop.hbase.io.ImmutableBytesWritable; @@ -146,7 +147,7 @@ public class ScanPerformanceEvaluation extends AbstractHBaseTool { Stopwatch scanTimer = new Stopwatch(); tableOpenTimer.start(); - HTable table = new HTable(getConf(), TableName.valueOf(tablename)); + Table table = new HTable(getConf(), TableName.valueOf(tablename)); tableOpenTimer.stop(); Scan scan = getScan(); diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/TestAcidGuarantees.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/TestAcidGuarantees.java index d73dd383b7a..172395a73be 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/TestAcidGuarantees.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/TestAcidGuarantees.java @@ -35,6 +35,7 @@ import org.apache.hadoop.hbase.client.Put; import org.apache.hadoop.hbase.client.Result; import org.apache.hadoop.hbase.client.ResultScanner; import org.apache.hadoop.hbase.client.Scan; +import org.apache.hadoop.hbase.client.Table; import org.apache.hadoop.hbase.regionserver.ConstantSizeRegionSplitPolicy; import org.apache.hadoop.hbase.util.Bytes; import org.apache.hadoop.util.StringUtils; @@ -97,7 +98,7 @@ public class TestAcidGuarantees implements Tool { byte data[] = new byte[10]; byte targetRows[][]; byte targetFamilies[][]; - HTable table; + Table table; AtomicLong numWritten = new AtomicLong(); public AtomicityWriter(TestContext ctx, byte targetRows[][], @@ -131,7 +132,7 @@ public class TestAcidGuarantees implements Tool { public static class AtomicGetReader extends RepeatingTestThread { byte targetRow[]; byte targetFamilies[][]; - HTable table; + Table table; int numVerified = 0; AtomicLong numRead = new AtomicLong(); @@ -189,7 +190,7 @@ public class TestAcidGuarantees implements Tool { */ public static class AtomicScanReader extends RepeatingTestThread { byte targetFamilies[][]; - HTable table; + Table table; AtomicLong numScans = new AtomicLong(); AtomicLong numRowsScanned = new AtomicLong(); diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/TestHBaseTestingUtility.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/TestHBaseTestingUtility.java index 8e448e84714..cb01558f10d 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/TestHBaseTestingUtility.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/TestHBaseTestingUtility.java @@ -28,9 +28,9 @@ import org.apache.commons.logging.LogFactory; import org.apache.hadoop.fs.FileSystem; import org.apache.hadoop.fs.Path; import org.apache.hadoop.hbase.client.Get; -import org.apache.hadoop.hbase.client.HTable; import org.apache.hadoop.hbase.client.Put; import org.apache.hadoop.hbase.client.Result; +import org.apache.hadoop.hbase.client.Table; import org.apache.hadoop.hbase.util.Bytes; import org.apache.hadoop.hbase.zookeeper.MiniZooKeeperCluster; import org.apache.hadoop.hdfs.MiniDFSCluster; @@ -87,8 +87,8 @@ public class TestHBaseTestingUtility { final byte[] QUAL_NAME = Bytes.toBytes("qual"); final byte[] VALUE = Bytes.toBytes("value"); - HTable table1 = htu1.createTable(TABLE_NAME, FAM_NAME); - HTable table2 = htu2.createTable(TABLE_NAME, FAM_NAME); + Table table1 = htu1.createTable(TABLE_NAME, FAM_NAME); + Table table2 = htu2.createTable(TABLE_NAME, FAM_NAME); Put put = new Put(ROW); put.add(FAM_NAME, QUAL_NAME, VALUE); diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/TestMetaTableAccessor.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/TestMetaTableAccessor.java index 1448bf2cd68..aec8bdc7ac2 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/TestMetaTableAccessor.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/TestMetaTableAccessor.java @@ -36,6 +36,7 @@ import org.apache.hadoop.hbase.client.HConnection; import org.apache.hadoop.hbase.client.HConnectionManager; import org.apache.hadoop.hbase.client.HTable; import org.apache.hadoop.hbase.client.Result; +import org.apache.hadoop.hbase.client.Table; import org.apache.hadoop.hbase.util.Bytes; import org.apache.hadoop.hbase.util.Pair; import org.junit.AfterClass; @@ -332,7 +333,7 @@ public class TestMetaTableAccessor { long seqNum100 = random.nextLong(); - HTable meta = MetaTableAccessor.getMetaHTable(hConnection); + Table meta = MetaTableAccessor.getMetaHTable(hConnection); try { MetaTableAccessor.updateRegionLocation(hConnection, primary, serverName0, seqNum0); @@ -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 { Get get = new Get(row); Result result = meta.get(get); diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/TestMultiVersions.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/TestMultiVersions.java index c04edc1c238..e619d5e5c28 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/TestMultiVersions.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/TestMultiVersions.java @@ -33,6 +33,7 @@ import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.hbase.HBaseTestCase.FlushCache; import org.apache.hadoop.hbase.HBaseTestCase.HTableIncommon; import org.apache.hadoop.hbase.HBaseTestCase.Incommon; +import org.apache.hadoop.hbase.client.Admin; import org.apache.hadoop.hbase.client.Get; import org.apache.hadoop.hbase.client.HBaseAdmin; import org.apache.hadoop.hbase.client.HTable; @@ -40,6 +41,7 @@ import org.apache.hadoop.hbase.client.Put; import org.apache.hadoop.hbase.client.Result; import org.apache.hadoop.hbase.client.ResultScanner; import org.apache.hadoop.hbase.client.Scan; +import org.apache.hadoop.hbase.client.Table; import org.apache.hadoop.hbase.util.Bytes; import org.junit.After; import org.junit.AfterClass; @@ -56,7 +58,7 @@ import org.junit.experimental.categories.Category; public class TestMultiVersions { private static final Log LOG = LogFactory.getLog(TestMultiVersions.class); private static final HBaseTestingUtility UTIL = new HBaseTestingUtility(); - private HBaseAdmin admin; + private Admin admin; private static final int NUM_SLAVES = 3; @@ -97,7 +99,7 @@ public class TestMultiVersions { hcd.setMaxVersions(3); desc.addFamily(hcd); this.admin.createTable(desc); - HTable table = new HTable(UTIL.getConfiguration(), desc.getTableName()); + Table table = new HTable(UTIL.getConfiguration(), desc.getTableName()); // TODO: Remove these deprecated classes or pull them in here if this is // only test using them. Incommon incommon = new HTableIncommon(table); @@ -140,7 +142,7 @@ public class TestMultiVersions { this.admin.createTable(desc); Put put = new Put(row, timestamp1); put.add(contents, contents, value1); - HTable table = new HTable(UTIL.getConfiguration(), tableName); + Table table = new HTable(UTIL.getConfiguration(), tableName); table.put(put); // Shut down and restart the HBase cluster table.close(); diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/TestNamespace.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/TestNamespace.java index a752f7aa6c9..79166a4a172 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/TestNamespace.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/TestNamespace.java @@ -37,6 +37,7 @@ import org.apache.hadoop.hbase.client.Admin; import org.apache.hadoop.hbase.client.Get; import org.apache.hadoop.hbase.client.HTable; import org.apache.hadoop.hbase.client.Put; +import org.apache.hadoop.hbase.client.Table; import org.apache.hadoop.hbase.master.HMaster; import org.apache.hadoop.hbase.util.Bytes; import org.apache.hadoop.hbase.util.FSUtils; @@ -249,7 +250,7 @@ public class TestNamespace { } //sanity check try to write and read from table - HTable table = new HTable(TEST_UTIL.getConfiguration(), desc.getTableName()); + Table table = new HTable(TEST_UTIL.getConfiguration(), desc.getTableName()); Put p = new Put(Bytes.toBytes("row1")); p.add(Bytes.toBytes("my_cf"),Bytes.toBytes("my_col"),Bytes.toBytes("value1")); table.put(p); diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/TestRegionRebalancing.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/TestRegionRebalancing.java index fdcb00c769b..fd7cd6f43ea 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/TestRegionRebalancing.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/TestRegionRebalancing.java @@ -29,8 +29,10 @@ import java.util.List; import org.apache.commons.logging.Log; import org.apache.commons.logging.LogFactory; +import org.apache.hadoop.hbase.client.Admin; import org.apache.hadoop.hbase.client.HBaseAdmin; import org.apache.hadoop.hbase.client.HTable; +import org.apache.hadoop.hbase.client.RegionLocator; import org.apache.hadoop.hbase.master.RegionStates; import org.apache.hadoop.hbase.protobuf.ProtobufUtil; import org.apache.hadoop.hbase.regionserver.HRegionServer; @@ -63,7 +65,7 @@ public class TestRegionRebalancing { private static final byte[] FAMILY_NAME = Bytes.toBytes("col"); public static final Log LOG = LogFactory.getLog(TestRegionRebalancing.class); private final HBaseTestingUtility UTIL = new HBaseTestingUtility(); - private HTable table; + private RegionLocator table; private HTableDescriptor desc; private String balancerName; @@ -94,7 +96,7 @@ public class TestRegionRebalancing { @Test (timeout=300000) public void testRebalanceOnRegionServerNumberChange() throws IOException, InterruptedException { - HBaseAdmin admin = new HBaseAdmin(UTIL.getConfiguration()); + Admin admin = new HBaseAdmin(UTIL.getConfiguration()); admin.createTable(this.desc, Arrays.copyOfRange(HBaseTestingUtility.KEYS, 1, HBaseTestingUtility.KEYS.length)); this.table = new HTable(UTIL.getConfiguration(), this.desc.getTableName()); diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/TestZooKeeper.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/TestZooKeeper.java index e65430b35ef..877b7273c89 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/TestZooKeeper.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/TestZooKeeper.java @@ -34,6 +34,7 @@ import java.util.Map; import org.apache.commons.logging.Log; import org.apache.commons.logging.LogFactory; import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.hbase.client.Admin; import org.apache.hadoop.hbase.client.Get; import org.apache.hadoop.hbase.client.HBaseAdmin; import org.apache.hadoop.hbase.client.HConnection; @@ -43,6 +44,7 @@ import org.apache.hadoop.hbase.client.Put; import org.apache.hadoop.hbase.client.Result; import org.apache.hadoop.hbase.client.ResultScanner; import org.apache.hadoop.hbase.client.Scan; +import org.apache.hadoop.hbase.client.Table; import org.apache.hadoop.hbase.master.HMaster; import org.apache.hadoop.hbase.master.LoadBalancer; import org.apache.hadoop.hbase.master.balancer.SimpleLoadBalancer; @@ -250,14 +252,14 @@ public class TestZooKeeper { HColumnDescriptor family = new HColumnDescriptor("fam"); desc.addFamily(family); LOG.info("Creating table " + tableName); - HBaseAdmin admin = new HBaseAdmin(TEST_UTIL.getConfiguration()); + Admin admin = new HBaseAdmin(TEST_UTIL.getConfiguration()); try { admin.createTable(desc); } finally { admin.close(); } - HTable table = + Table table = new HTable(new Configuration(TEST_UTIL.getConfiguration()), tableName); Put put = new Put(Bytes.toBytes("testrow")); put.add(Bytes.toBytes("fam"), @@ -270,11 +272,11 @@ public class TestZooKeeper { @Test public void testMultipleZK() throws IOException, NoSuchMethodException, InvocationTargetException, IllegalAccessException { - HTable localMeta = + Table localMeta = new HTable(new Configuration(TEST_UTIL.getConfiguration()), TableName.META_TABLE_NAME); Configuration otherConf = new Configuration(TEST_UTIL.getConfiguration()); otherConf.set(HConstants.ZOOKEEPER_QUORUM, "127.0.0.1"); - HTable ipMeta = new HTable(otherConf, TableName.META_TABLE_NAME); + Table ipMeta = new HTable(otherConf, TableName.META_TABLE_NAME); // dummy, just to open the connection final byte [] row = new byte [] {'r'}; @@ -491,7 +493,7 @@ public class TestZooKeeper { ZooKeeperWatcher zkw = m.getZooKeeper(); int expectedNumOfListeners = zkw.getNumberOfListeners(); // now the cluster is up. So assign some regions. - HBaseAdmin admin = new HBaseAdmin(TEST_UTIL.getConfiguration()); + Admin admin = new HBaseAdmin(TEST_UTIL.getConfiguration()); try { byte[][] SPLIT_KEYS = new byte[][] { Bytes.toBytes("a"), Bytes.toBytes("b"), Bytes.toBytes("c"), Bytes.toBytes("d"), Bytes.toBytes("e"), Bytes.toBytes("f"), @@ -530,8 +532,8 @@ public class TestZooKeeper { cluster.startRegionServer(); HMaster m = cluster.getMaster(); // now the cluster is up. So assign some regions. - HBaseAdmin admin = new HBaseAdmin(TEST_UTIL.getConfiguration()); - HTable table = null; + Admin admin = new HBaseAdmin(TEST_UTIL.getConfiguration()); + Table table = null; try { byte[][] SPLIT_KEYS = new byte[][] { Bytes.toBytes("1"), Bytes.toBytes("2"), Bytes.toBytes("3"), Bytes.toBytes("4"), Bytes.toBytes("5") }; diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAdmin.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAdmin.java index 664ae04f383..b91c9625098 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAdmin.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAdmin.java @@ -246,7 +246,7 @@ public class TestAdmin { final byte [] qualifier = Bytes.toBytes("qualifier"); final byte [] value = Bytes.toBytes("value"); final TableName table = TableName.valueOf("testDisableAndEnableTable"); - HTable ht = TEST_UTIL.createTable(table, HConstants.CATALOG_FAMILY); + Table ht = TEST_UTIL.createTable(table, HConstants.CATALOG_FAMILY); Put put = new Put(row); put.add(HConstants.CATALOG_FAMILY, qualifier, value); ht.put(put); @@ -291,8 +291,8 @@ public class TestAdmin { final byte [] value = Bytes.toBytes("value"); final byte [] table1 = Bytes.toBytes("testDisableAndEnableTable1"); final byte [] table2 = Bytes.toBytes("testDisableAndEnableTable2"); - HTable ht1 = TEST_UTIL.createTable(table1, HConstants.CATALOG_FAMILY); - HTable ht2 = TEST_UTIL.createTable(table2, HConstants.CATALOG_FAMILY); + Table ht1 = TEST_UTIL.createTable(table1, HConstants.CATALOG_FAMILY); + Table ht2 = TEST_UTIL.createTable(table2, HConstants.CATALOG_FAMILY); Put put = new Put(row); put.add(HConstants.CATALOG_FAMILY, qualifier, value); ht1.put(put); @@ -400,7 +400,7 @@ public class TestAdmin { htd.addFamily(fam2); htd.addFamily(fam3); this.admin.createTable(htd); - HTable table = new HTable(TEST_UTIL.getConfiguration(), "myTestTable"); + Table table = new HTable(TEST_UTIL.getConfiguration(), "myTestTable"); HTableDescriptor confirmedHtd = table.getTableDescriptor(); assertEquals(htd.compareTo(confirmedHtd), 0); table.close(); @@ -816,7 +816,7 @@ public class TestAdmin { TableName TABLE_4 = TableName.valueOf(tableName.getNameAsString() + "_4"); desc = new HTableDescriptor(TABLE_4); desc.addFamily(new HColumnDescriptor(HConstants.CATALOG_FAMILY)); - HBaseAdmin ladmin = new HBaseAdmin(TEST_UTIL.getConfiguration()); + Admin ladmin = new HBaseAdmin(TEST_UTIL.getConfiguration()); try { ladmin.createTable(desc, splitKeys); assertTrue("Should not be able to create this table because of " + @@ -1238,7 +1238,7 @@ public class TestAdmin { // Use 80 bit numbers to make sure we aren't limited byte [] startKey = { 1, 1, 1, 1, 1, 1, 1, 1, 1, 1 }; byte [] endKey = { 9, 9, 9, 9, 9, 9, 9, 9, 9, 9 }; - HBaseAdmin hbaseadmin = new HBaseAdmin(TEST_UTIL.getConfiguration()); + Admin hbaseadmin = new HBaseAdmin(TEST_UTIL.getConfiguration()); HTableDescriptor htd = new HTableDescriptor(TableName.valueOf(name)); htd.addFamily(new HColumnDescriptor(HConstants.CATALOG_FAMILY)); hbaseadmin.createTable(htd, startKey, endKey, expectedRegions); @@ -1255,7 +1255,7 @@ public class TestAdmin { @Test (timeout=300000) public void testReadOnlyTable() throws Exception { byte [] name = Bytes.toBytes("testReadOnlyTable"); - HTable table = TEST_UTIL.createTable(name, HConstants.CATALOG_FAMILY); + Table table = TEST_UTIL.createTable(name, HConstants.CATALOG_FAMILY); byte[] value = Bytes.toBytes("somedata"); // This used to use an empty row... That must have been a bug Put put = new Put(value); @@ -1324,7 +1324,7 @@ public class TestAdmin { @Test (expected=TableNotDisabledException.class, timeout=300000) public void testTableNotDisabledExceptionWithATable() throws IOException { final TableName name = TableName.valueOf("testTableNotDisabledExceptionWithATable"); - HTable t = TEST_UTIL.createTable(name, HConstants.CATALOG_FAMILY); + Table t = TEST_UTIL.createTable(name, HConstants.CATALOG_FAMILY); try { this.admin.enableTable(name); }finally { @@ -1338,7 +1338,7 @@ public class TestAdmin { */ @Test (expected=TableNotFoundException.class, timeout=300000) public void testTableNotFoundExceptionWithoutAnyTables() throws IOException { - HTable ht = + Table ht = new HTable(TEST_UTIL.getConfiguration(),"testTableNotFoundExceptionWithoutAnyTables"); ht.get(new Get("e".getBytes())); } @@ -1659,7 +1659,7 @@ public class TestAdmin { HTableDescriptor desc = new HTableDescriptor(TableName.valueOf(tableName)); desc.addFamily(new HColumnDescriptor(HConstants.CATALOG_FAMILY)); admin.createTable(desc); - HTable table = new HTable(TEST_UTIL.getConfiguration(), tableName); + Table table = new HTable(TEST_UTIL.getConfiguration(), tableName); HRegionServer regionServer = TEST_UTIL.getRSForFirstRegionInTable(Bytes.toBytes(tableName)); for (int i = 1; i <= 256; i++) { // 256 writes should cause 8 log rolls diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestClientOperationInterrupt.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestClientOperationInterrupt.java index bcae50898f8..b29456f47f0 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestClientOperationInterrupt.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestClientOperationInterrupt.java @@ -83,7 +83,7 @@ public class TestClientOperationInterrupt { } util.createTable(tableName, new byte[][]{dummy, test}); - HTable ht = new HTable(conf, tableName); + Table ht = new HTable(conf, tableName); Put p = new Put(row1); p.add(dummy, dummy, dummy); ht.put(p); @@ -105,7 +105,7 @@ public class TestClientOperationInterrupt { @Override public void run() { try { - HTable ht = new HTable(conf, tableName); + Table ht = new HTable(conf, tableName); Result r = ht.get(new Get(row1)); noEx.incrementAndGet(); } catch (IOException e) { @@ -154,7 +154,7 @@ public class TestClientOperationInterrupt { Thread.sleep(1); } - HTable ht = new HTable(conf, tableName); + Table ht = new HTable(conf, tableName); Result r = ht.get(new Get(row1)); Assert.assertFalse(r.isEmpty()); } diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestClientScannerRPCTimeout.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestClientScannerRPCTimeout.java index 95b9b215564..546b9ea965c 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestClientScannerRPCTimeout.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestClientScannerRPCTimeout.java @@ -84,7 +84,7 @@ public class TestClientScannerRPCTimeout { @Test public void testScannerNextRPCTimesout() throws Exception { final byte[] TABLE_NAME = Bytes.toBytes("testScannerNextRPCTimesout"); - HTable ht = TEST_UTIL.createTable(TABLE_NAME, FAMILY); + Table ht = TEST_UTIL.createTable(TABLE_NAME, FAMILY); byte[] r1 = Bytes.toBytes("row-1"); byte[] r2 = Bytes.toBytes("row-2"); byte[] r3 = Bytes.toBytes("row-3"); @@ -123,7 +123,7 @@ public class TestClientScannerRPCTimeout { RSRpcServicesWithScanTimeout.tryNumber <= CLIENT_RETRIES_NUMBER); } - private void putToTable(HTable ht, byte[] rowkey) throws IOException { + private void putToTable(Table ht, byte[] rowkey) throws IOException { Put put = new Put(rowkey); put.add(FAMILY, QUALIFIER, VALUE); ht.put(put); diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestClientTimeouts.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestClientTimeouts.java index 1d6ef77e21a..0bb73ca6f8d 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestClientTimeouts.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestClientTimeouts.java @@ -87,7 +87,7 @@ public class TestClientTimeouts { // Ensure the HBaseAdmin uses a new connection by changing Configuration. Configuration conf = HBaseConfiguration.create(TEST_UTIL.getConfiguration()); conf.set(HConstants.HBASE_CLIENT_INSTANCE_ID, String.valueOf(-1)); - HBaseAdmin admin = null; + Admin admin = null; try { admin = new HBaseAdmin(conf); HConnection connection = admin.getConnection(); diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestFromClientSide.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestFromClientSide.java index 0916e3837f4..3b53c0aeefa 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestFromClientSide.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestFromClientSide.java @@ -184,7 +184,7 @@ public class TestFromClientSide { desc.addFamily(hcd); TEST_UTIL.getHBaseAdmin().createTable(desc); Configuration c = TEST_UTIL.getConfiguration(); - HTable h = new HTable(c, TABLENAME); + Table h = new HTable(c, TABLENAME); long ts = System.currentTimeMillis(); Put p = new Put(T1, ts); @@ -244,7 +244,7 @@ public class TestFromClientSide { final byte[] COLUMN = Bytes.toBytes("column"); final byte[] VALUE = Bytes.toBytes("value"); - HTable table = TEST_UTIL.createTable(TABLENAME, FAMILY); + Table table = TEST_UTIL.createTable(TABLENAME, FAMILY); // future timestamp long ts = System.currentTimeMillis() * 2; @@ -386,7 +386,7 @@ public class TestFromClientSide { byte[] TABLE = Bytes.toBytes("testGetConfiguration"); byte[][] FAMILIES = new byte[][] { Bytes.toBytes("foo") }; Configuration conf = TEST_UTIL.getConfiguration(); - HTable table = TEST_UTIL.createTable(TABLE, FAMILIES, conf); + Table table = TEST_UTIL.createTable(TABLE, FAMILIES, conf); assertSame(conf, table.getConfiguration()); } @@ -415,7 +415,7 @@ public class TestFromClientSide { putRows(ht, 3, value2, keyPrefix1); putRows(ht, 3, value2, keyPrefix2); putRows(ht, 3, value2, keyPrefix3); - HTable table = new HTable(TEST_UTIL.getConfiguration(), TABLE); + Table table = new HTable(TEST_UTIL.getConfiguration(), TABLE); System.out.println("Checking values for key: " + keyPrefix1); assertEquals("Got back incorrect number of rows from scan", 3, getNumberOfRows(keyPrefix1, value2, table)); @@ -442,7 +442,7 @@ public class TestFromClientSide { getNumberOfRows(keyPrefix2, value2, table)); } - private void deleteColumns(HTable ht, String value, String keyPrefix) + private void deleteColumns(Table ht, String value, String keyPrefix) throws IOException { ResultScanner scanner = buildScanner(keyPrefix, value, ht); Iterator it = scanner.iterator(); @@ -457,7 +457,7 @@ public class TestFromClientSide { assertEquals("Did not perform correct number of deletes", 3, count); } - private int getNumberOfRows(String keyPrefix, String value, HTable ht) + private int getNumberOfRows(String keyPrefix, String value, Table ht) throws Exception { ResultScanner resultScanner = buildScanner(keyPrefix, value, ht); Iterator scanner = resultScanner.iterator(); @@ -474,7 +474,7 @@ public class TestFromClientSide { return numberOfResults; } - private ResultScanner buildScanner(String keyPrefix, String value, HTable ht) + private ResultScanner buildScanner(String keyPrefix, String value, Table ht) throws IOException { // OurFilterList allFilters = new OurFilterList(); FilterList allFilters = new FilterList(/* FilterList.Operator.MUST_PASS_ALL */); @@ -500,7 +500,7 @@ public class TestFromClientSide { return ht.getScanner(scan); } - private void putRows(HTable ht, int numRows, String value, String key) + private void putRows(Table ht, int numRows, String value, String key) throws IOException { for (int i = 0; i < numRows; i++) { String row = key + "_" + UUID.randomUUID().toString(); @@ -614,7 +614,7 @@ public class TestFromClientSide { * @return Count of rows in table. * @throws IOException */ - private int countRows(final HTable t, final Scan s) + private int countRows(final Table t, final Scan s) throws IOException { // Assert all rows in table. ResultScanner scanner = t.getScanner(s); @@ -627,7 +627,7 @@ public class TestFromClientSide { return count; } - private void assertRowCount(final HTable t, final int expected) + private void assertRowCount(final Table t, final int expected) throws IOException { assertEquals(expected, countRows(t, new Scan())); } @@ -675,7 +675,7 @@ public class TestFromClientSide { @Test public void testSuperSimple() throws Exception { byte [] TABLE = Bytes.toBytes("testSuperSimple"); - HTable ht = TEST_UTIL.createTable(TABLE, FAMILY); + Table ht = TEST_UTIL.createTable(TABLE, FAMILY); Put put = new Put(ROW); put.add(FAMILY, QUALIFIER, VALUE); ht.put(put); @@ -692,7 +692,7 @@ public class TestFromClientSide { byte [] TABLE = Bytes.toBytes("testMaxKeyValueSize"); Configuration conf = TEST_UTIL.getConfiguration(); String oldMaxSize = conf.get("hbase.client.keyvalue.maxsize"); - HTable ht = TEST_UTIL.createTable(TABLE, FAMILY); + Table ht = TEST_UTIL.createTable(TABLE, FAMILY); byte[] value = new byte[4 * 1024 * 1024]; Put put = new Put(ROW); put.add(FAMILY, QUALIFIER, value); @@ -712,7 +712,7 @@ public class TestFromClientSide { @Test public void testFilters() throws Exception { byte [] TABLE = Bytes.toBytes("testFilters"); - HTable ht = TEST_UTIL.createTable(TABLE, FAMILY); + Table ht = TEST_UTIL.createTable(TABLE, FAMILY); byte [][] ROWS = makeN(ROW, 10); byte [][] QUALIFIERS = { Bytes.toBytes("col0--"), Bytes.toBytes("col1--"), @@ -748,7 +748,7 @@ public class TestFromClientSide { @Test public void testKeyOnlyFilter() throws Exception { byte [] TABLE = Bytes.toBytes("testKeyOnlyFilter"); - HTable ht = TEST_UTIL.createTable(TABLE, FAMILY); + Table ht = TEST_UTIL.createTable(TABLE, FAMILY); byte [][] ROWS = makeN(ROW, 10); byte [][] QUALIFIERS = { Bytes.toBytes("col0--"), Bytes.toBytes("col1--"), @@ -785,7 +785,7 @@ public class TestFromClientSide { @Test public void testSimpleMissing() throws Exception { byte [] TABLE = Bytes.toBytes("testSimpleMissing"); - HTable ht = TEST_UTIL.createTable(TABLE, FAMILY); + Table ht = TEST_UTIL.createTable(TABLE, FAMILY); byte [][] ROWS = makeN(ROW, 4); // Try to get a row on an empty table @@ -901,7 +901,7 @@ public class TestFromClientSide { byte [][] QUALIFIERS = makeN(QUALIFIER, 10); byte [][] VALUES = makeN(VALUE, 10); - HTable ht = TEST_UTIL.createTable(TABLE, FAMILIES); + Table ht = TEST_UTIL.createTable(TABLE, FAMILIES); Get get; Scan scan; @@ -1210,7 +1210,7 @@ public class TestFromClientSide { fail("Creating a table with a null family passed, should fail"); } catch(Exception e) {} - HTable ht = TEST_UTIL.createTable(TABLE, FAMILY); + Table ht = TEST_UTIL.createTable(TABLE, FAMILY); // Null row (should NOT work) try { @@ -1309,7 +1309,7 @@ public class TestFromClientSide { long [] STAMPS = makeStamps(20); byte [][] VALUES = makeNAscii(VALUE, 20); - HTable ht = TEST_UTIL.createTable(TABLE, FAMILY, 10); + Table ht = TEST_UTIL.createTable(TABLE, FAMILY, 10); // Insert 4 versions of same column Put put = new Put(ROW); @@ -1523,7 +1523,7 @@ public class TestFromClientSide { int [] LIMITS = {1,3,5}; long [] STAMPS = makeStamps(10); byte [][] VALUES = makeNAscii(VALUE, 10); - HTable ht = TEST_UTIL.createTable(TABLE, FAMILIES, LIMITS); + Table ht = TEST_UTIL.createTable(TABLE, FAMILIES, LIMITS); // Insert limit + 1 on each family Put put = new Put(ROW); @@ -1719,7 +1719,7 @@ public class TestFromClientSide { byte [][] VALUES = makeN(VALUE, 5); long [] ts = {1000, 2000, 3000, 4000, 5000}; - HTable ht = TEST_UTIL.createTable(TABLE, FAMILY, 5); + Table ht = TEST_UTIL.createTable(TABLE, FAMILY, 5); Put put = new Put(ROW); for (int q = 0; q < 1; q++) @@ -1758,7 +1758,7 @@ public class TestFromClientSide { long [] ts = {1000, 2000, 3000, 4000, 5000}; HBaseAdmin admin = new HBaseAdmin(TEST_UTIL.getConfiguration()); - HTable ht = TEST_UTIL.createTable(TABLE, FAMILY, 5); + Table ht = TEST_UTIL.createTable(TABLE, FAMILY, 5); Put put = null; Result result = null; Get get = null; @@ -1873,7 +1873,7 @@ public class TestFromClientSide { byte [][] VALUES = makeN(VALUE, 5); long [] ts = {1000, 2000, 3000, 4000, 5000}; - HTable ht = TEST_UTIL.createTable(TABLE, FAMILIES, 3); + Table ht = TEST_UTIL.createTable(TABLE, FAMILIES, 3); Put put = new Put(ROW); put.add(FAMILIES[0], QUALIFIER, ts[0], VALUES[0]); @@ -2240,7 +2240,7 @@ public class TestFromClientSide { byte [][] ROWS = makeN(ROW, numRows); byte [][] QUALIFIERS = makeN(QUALIFIER, numColsPerRow); - HTable ht = TEST_UTIL.createTable(TABLE, FAMILY); + Table ht = TEST_UTIL.createTable(TABLE, FAMILY); // Insert rows @@ -2323,7 +2323,7 @@ public class TestFromClientSide { byte [][] VALUES = makeNAscii(VALUE, 7); long [] STAMPS = makeStamps(7); - HTable ht = TEST_UTIL.createTable(TABLE, FAMILY, 10); + Table ht = TEST_UTIL.createTable(TABLE, FAMILY, 10); // Insert three versions @@ -2387,7 +2387,7 @@ public class TestFromClientSide { byte [][] VALUES = makeNAscii(VALUE, 7); long [] STAMPS = makeStamps(7); - HTable ht = TEST_UTIL.createTable(TABLE, FAMILY, 10); + Table ht = TEST_UTIL.createTable(TABLE, FAMILY, 10); // Insert lots versions @@ -2434,7 +2434,7 @@ public class TestFromClientSide { byte [] TABLE = Bytes.toBytes("testJiraTest1014"); - HTable ht = TEST_UTIL.createTable(TABLE, FAMILY, 10); + Table ht = TEST_UTIL.createTable(TABLE, FAMILY, 10); long manualStamp = 12345; @@ -2461,7 +2461,7 @@ public class TestFromClientSide { byte [][] VALUES = makeNAscii(VALUE, 7); long [] STAMPS = makeStamps(7); - HTable ht = TEST_UTIL.createTable(TABLE, FAMILY, 10); + Table ht = TEST_UTIL.createTable(TABLE, FAMILY, 10); // Insert lots versions @@ -2504,7 +2504,7 @@ public class TestFromClientSide { byte [][] VALUES = makeNAscii(VALUE, 7); long [] STAMPS = makeStamps(7); - HTable ht = TEST_UTIL.createTable(TABLE, FAMILY, 10); + Table ht = TEST_UTIL.createTable(TABLE, FAMILY, 10); // Insert lots versions @@ -2533,7 +2533,7 @@ public class TestFromClientSide { // Bulk Testers // - private void getVersionRangeAndVerifyGreaterThan(HTable ht, byte [] row, + private void getVersionRangeAndVerifyGreaterThan(Table ht, byte [] row, byte [] family, byte [] qualifier, long [] stamps, byte [][] values, int start, int end) throws IOException { @@ -2545,7 +2545,7 @@ public class TestFromClientSide { assertNResult(result, row, family, qualifier, stamps, values, start+1, end); } - private void getVersionRangeAndVerify(HTable ht, byte [] row, byte [] family, + private void getVersionRangeAndVerify(Table ht, byte [] row, byte [] family, byte [] qualifier, long [] stamps, byte [][] values, int start, int end) throws IOException { Get get = new Get(row); @@ -2556,7 +2556,7 @@ public class TestFromClientSide { assertNResult(result, row, family, qualifier, stamps, values, start, end); } - private void getAllVersionsAndVerify(HTable ht, byte [] row, byte [] family, + private void getAllVersionsAndVerify(Table ht, byte [] row, byte [] family, byte [] qualifier, long [] stamps, byte [][] values, int start, int end) throws IOException { Get get = new Get(row); @@ -2566,7 +2566,7 @@ public class TestFromClientSide { assertNResult(result, row, family, qualifier, stamps, values, start, end); } - private void scanVersionRangeAndVerifyGreaterThan(HTable ht, byte [] row, + private void scanVersionRangeAndVerifyGreaterThan(Table ht, byte [] row, byte [] family, byte [] qualifier, long [] stamps, byte [][] values, int start, int end) throws IOException { @@ -2578,7 +2578,7 @@ public class TestFromClientSide { assertNResult(result, row, family, qualifier, stamps, values, start+1, end); } - private void scanVersionRangeAndVerify(HTable ht, byte [] row, byte [] family, + private void scanVersionRangeAndVerify(Table ht, byte [] row, byte [] family, byte [] qualifier, long [] stamps, byte [][] values, int start, int end) throws IOException { Scan scan = new Scan(row); @@ -2589,7 +2589,7 @@ public class TestFromClientSide { assertNResult(result, row, family, qualifier, stamps, values, start, end); } - private void scanAllVersionsAndVerify(HTable ht, byte [] row, byte [] family, + private void scanAllVersionsAndVerify(Table ht, byte [] row, byte [] family, byte [] qualifier, long [] stamps, byte [][] values, int start, int end) throws IOException { Scan scan = new Scan(row); @@ -2599,7 +2599,7 @@ public class TestFromClientSide { assertNResult(result, row, family, qualifier, stamps, values, start, end); } - private void getVersionAndVerify(HTable ht, byte [] row, byte [] family, + private void getVersionAndVerify(Table ht, byte [] row, byte [] family, byte [] qualifier, long stamp, byte [] value) throws Exception { Get get = new Get(row); @@ -2610,7 +2610,7 @@ public class TestFromClientSide { assertSingleResult(result, row, family, qualifier, stamp, value); } - private void getVersionAndVerifyMissing(HTable ht, byte [] row, byte [] family, + private void getVersionAndVerifyMissing(Table ht, byte [] row, byte [] family, byte [] qualifier, long stamp) throws Exception { Get get = new Get(row); @@ -2621,7 +2621,7 @@ public class TestFromClientSide { assertEmptyResult(result); } - private void scanVersionAndVerify(HTable ht, byte [] row, byte [] family, + private void scanVersionAndVerify(Table ht, byte [] row, byte [] family, byte [] qualifier, long stamp, byte [] value) throws Exception { Scan scan = new Scan(row); @@ -2632,7 +2632,7 @@ public class TestFromClientSide { assertSingleResult(result, row, family, qualifier, stamp, value); } - private void scanVersionAndVerifyMissing(HTable ht, byte [] row, + private void scanVersionAndVerifyMissing(Table ht, byte [] row, byte [] family, byte [] qualifier, long stamp) throws Exception { Scan scan = new Scan(row); @@ -2643,7 +2643,7 @@ public class TestFromClientSide { assertNullResult(result); } - private void getTestNull(HTable ht, byte [] row, byte [] family, + private void getTestNull(Table ht, byte [] row, byte [] family, byte [] value) throws Exception { @@ -2668,12 +2668,12 @@ public class TestFromClientSide { } - private void scanTestNull(HTable ht, byte[] row, byte[] family, byte[] value) + private void scanTestNull(Table ht, byte[] row, byte[] family, byte[] value) throws Exception { scanTestNull(ht, row, family, value, false); } - private void scanTestNull(HTable ht, byte[] row, byte[] family, byte[] value, + private void scanTestNull(Table ht, byte[] row, byte[] family, byte[] value, boolean isReversedScan) throws Exception { Scan scan = new Scan(); @@ -2701,7 +2701,7 @@ public class TestFromClientSide { } - private void singleRowGetTest(HTable ht, byte [][] ROWS, byte [][] FAMILIES, + private void singleRowGetTest(Table ht, byte [][] ROWS, byte [][] FAMILIES, byte [][] QUALIFIERS, byte [][] VALUES) throws Exception { @@ -2799,7 +2799,7 @@ public class TestFromClientSide { } - private void singleRowScanTest(HTable ht, byte [][] ROWS, byte [][] FAMILIES, + private void singleRowScanTest(Table ht, byte [][] ROWS, byte [][] FAMILIES, byte [][] QUALIFIERS, byte [][] VALUES) throws Exception { @@ -2901,7 +2901,7 @@ public class TestFromClientSide { * Expects family and qualifier arrays to be valid for at least * the range: idx-2 < idx < idx+2 */ - private void getVerifySingleColumn(HTable ht, + private void getVerifySingleColumn(Table ht, byte [][] ROWS, int ROWIDX, byte [][] FAMILIES, int FAMILYIDX, byte [][] QUALIFIERS, int QUALIFIERIDX, @@ -2959,7 +2959,7 @@ public class TestFromClientSide { * the range: idx-2 to idx+2 * Expects row array to be valid for at least idx to idx+2 */ - private void scanVerifySingleColumn(HTable ht, + private void scanVerifySingleColumn(Table ht, byte [][] ROWS, int ROWIDX, byte [][] FAMILIES, int FAMILYIDX, byte [][] QUALIFIERS, int QUALIFIERIDX, @@ -3019,7 +3019,7 @@ public class TestFromClientSide { * Verify we do not read any values by accident around a single column * Same requirements as getVerifySingleColumn */ - private void getVerifySingleEmpty(HTable ht, + private void getVerifySingleEmpty(Table ht, byte [][] ROWS, int ROWIDX, byte [][] FAMILIES, int FAMILYIDX, byte [][] QUALIFIERS, int QUALIFIERIDX) @@ -3050,7 +3050,7 @@ public class TestFromClientSide { } - private void scanVerifySingleEmpty(HTable ht, + private void scanVerifySingleEmpty(Table ht, byte [][] ROWS, int ROWIDX, byte [][] FAMILIES, int FAMILYIDX, byte [][] QUALIFIERS, int QUALIFIERIDX) @@ -3275,7 +3275,7 @@ public class TestFromClientSide { // Helpers // - private Result getSingleScanResult(HTable ht, Scan scan) throws IOException { + private Result getSingleScanResult(Table ht, Scan scan) throws IOException { ResultScanner scanner = ht.getScanner(scan); Result result = scanner.next(); scanner.close(); @@ -3335,7 +3335,7 @@ public class TestFromClientSide { long [] STAMPS = makeStamps(20); byte [][] VALUES = makeNAscii(VALUE, 20); - HTable ht = TEST_UTIL.createTable(TABLE, FAMILY, 10); + Table ht = TEST_UTIL.createTable(TABLE, FAMILY, 10); // Insert 4 versions of same column Put put = new Put(ROW); @@ -3546,7 +3546,7 @@ public class TestFromClientSide { public void testUpdates() throws Exception { byte [] TABLE = Bytes.toBytes("testUpdates"); - HTable hTable = TEST_UTIL.createTable(TABLE, FAMILY, 10); + Table hTable = TEST_UTIL.createTable(TABLE, FAMILY, 10); // Write a column with values at timestamp 1, 2 and 3 byte[] row = Bytes.toBytes("row1"); @@ -3597,7 +3597,7 @@ public class TestFromClientSide { String tableName = "testUpdatesWithMajorCompaction"; byte [] TABLE = Bytes.toBytes(tableName); - HTable hTable = TEST_UTIL.createTable(TABLE, FAMILY, 10); + Table hTable = TEST_UTIL.createTable(TABLE, FAMILY, 10); HBaseAdmin admin = new HBaseAdmin(TEST_UTIL.getConfiguration()); // Write a column with values at timestamp 1, 2 and 3 @@ -3659,7 +3659,7 @@ public class TestFromClientSide { String tableName = "testMajorCompactionBetweenTwoUpdates"; byte [] TABLE = Bytes.toBytes(tableName); - HTable hTable = TEST_UTIL.createTable(TABLE, FAMILY, 10); + Table hTable = TEST_UTIL.createTable(TABLE, FAMILY, 10); HBaseAdmin admin = new HBaseAdmin(TEST_UTIL.getConfiguration()); // Write a column with values at timestamp 1, 2 and 3 @@ -3724,7 +3724,7 @@ public class TestFromClientSide { @Test public void testGet_EmptyTable() throws IOException { - HTable table = TEST_UTIL.createTable(Bytes.toBytes("testGet_EmptyTable"), FAMILY); + Table table = TEST_UTIL.createTable(Bytes.toBytes("testGet_EmptyTable"), FAMILY); Get get = new Get(ROW); get.addFamily(FAMILY); Result r = table.get(get); @@ -3733,7 +3733,7 @@ public class TestFromClientSide { @Test public void testGet_NullQualifier() throws IOException { - HTable table = TEST_UTIL.createTable(Bytes.toBytes("testGet_NullQualifier"), FAMILY); + Table table = TEST_UTIL.createTable(Bytes.toBytes("testGet_NullQualifier"), FAMILY); Put put = new Put(ROW); put.add(FAMILY, QUALIFIER, VALUE); table.put(put); @@ -3756,7 +3756,7 @@ public class TestFromClientSide { @Test public void testGet_NonExistentRow() throws IOException { - HTable table = TEST_UTIL.createTable(Bytes.toBytes("testGet_NonExistentRow"), FAMILY); + Table table = TEST_UTIL.createTable(Bytes.toBytes("testGet_NonExistentRow"), FAMILY); Put put = new Put(ROW); put.add(FAMILY, QUALIFIER, VALUE); table.put(put); @@ -3783,7 +3783,7 @@ public class TestFromClientSide { final byte [] row1 = Bytes.toBytes("row1"); final byte [] row2 = Bytes.toBytes("row2"); final byte [] value = Bytes.toBytes("abcd"); - HTable table = TEST_UTIL.createTable(Bytes.toBytes("testPut"), + Table table = TEST_UTIL.createTable(Bytes.toBytes("testPut"), new byte [][] {CONTENTS_FAMILY, SMALL_FAMILY}); Put put = new Put(row1); put.add(CONTENTS_FAMILY, null, value); @@ -3820,7 +3820,7 @@ public class TestFromClientSide { public void testPutNoCF() throws IOException { final byte[] BAD_FAM = Bytes.toBytes("BAD_CF"); final byte[] VAL = Bytes.toBytes(100); - HTable table = TEST_UTIL.createTable(Bytes.toBytes("testPutNoCF"), new byte[][]{FAMILY}); + Table table = TEST_UTIL.createTable(Bytes.toBytes("testPutNoCF"), new byte[][]{FAMILY}); boolean caughtNSCFE = false; @@ -3841,7 +3841,7 @@ public class TestFromClientSide { final byte[] SMALL_FAMILY = Bytes.toBytes("smallfam"); final int NB_BATCH_ROWS = 10; final byte[] value = Bytes.toBytes("abcd"); - HTable table = TEST_UTIL.createTable(Bytes.toBytes("testRowsPut"), + Table table = TEST_UTIL.createTable(Bytes.toBytes("testRowsPut"), new byte[][] {CONTENTS_FAMILY, SMALL_FAMILY }); ArrayList rowsUpdate = new ArrayList(); for (int i = 0; i < NB_BATCH_ROWS; i++) { @@ -3974,7 +3974,7 @@ public class TestFromClientSide { final byte [] FAM1 = Bytes.toBytes("fam1"); final byte [] FAM2 = Bytes.toBytes("fam2"); // Open table - HTable table = TEST_UTIL.createTable(Bytes.toBytes("testHBase737"), + Table table = TEST_UTIL.createTable(Bytes.toBytes("testHBase737"), new byte [][] {FAM1, FAM2}); // Insert some values Put put = new Put(ROW); @@ -4069,7 +4069,7 @@ public class TestFromClientSide { for (int i = 0; i < tables.length; i++) { TEST_UTIL.createTable(tables[i], FAMILY); } - HBaseAdmin admin = new HBaseAdmin(TEST_UTIL.getConfiguration()); + Admin admin = new HBaseAdmin(TEST_UTIL.getConfiguration()); HTableDescriptor[] ts = admin.listTables(); HashSet result = new HashSet(ts.length); for (int i = 0; i < ts.length; i++) { @@ -4156,14 +4156,14 @@ public class TestFromClientSide { final byte[] attrValue = Bytes.toBytes("somevalue"); byte[] value = Bytes.toBytes("value"); - HTable a = TEST_UTIL.createTable(tableAname, HConstants.CATALOG_FAMILY); - HTable b = TEST_UTIL.createTable(tableBname, HConstants.CATALOG_FAMILY); + Table a = TEST_UTIL.createTable(tableAname, HConstants.CATALOG_FAMILY); + Table b = TEST_UTIL.createTable(tableBname, HConstants.CATALOG_FAMILY); Put put = new Put(ROW); put.add(HConstants.CATALOG_FAMILY, null, value); a.put(put); // open a new connection to A and a connection to b - HTable newA = new HTable(TEST_UTIL.getConfiguration(), tableAname); + Table newA = new HTable(TEST_UTIL.getConfiguration(), tableAname); // copy data from A to B Scan scan = new Scan(); @@ -4183,7 +4183,7 @@ public class TestFromClientSide { } // Opening a new connection to A will cause the tables to be reloaded - HTable anotherA = new HTable(TEST_UTIL.getConfiguration(), tableAname); + Table anotherA = new HTable(TEST_UTIL.getConfiguration(), tableAname); Get get = new Get(ROW); get.addFamily(HConstants.CATALOG_FAMILY); anotherA.get(get); @@ -4345,7 +4345,7 @@ public class TestFromClientSide { final byte [] TABLENAME = Bytes.toBytes("testMultiRowMutation"); final byte [] ROW1 = Bytes.toBytes("testRow1"); - HTable t = TEST_UTIL.createTable(TABLENAME, FAMILY); + Table t = TEST_UTIL.createTable(TABLENAME, FAMILY); Put p = new Put(ROW); p.add(FAMILY, QUALIFIER, VALUE); MutationProto m1 = ProtobufUtil.toMutation(MutationType.PUT, p); @@ -4374,7 +4374,7 @@ public class TestFromClientSide { public void testRowMutation() throws Exception { LOG.info("Starting testRowMutation"); final byte [] TABLENAME = Bytes.toBytes("testRowMutation"); - HTable t = TEST_UTIL.createTable(TABLENAME, FAMILY); + Table t = TEST_UTIL.createTable(TABLENAME, FAMILY); byte [][] QUALIFIERS = new byte [][] { Bytes.toBytes("a"), Bytes.toBytes("b") }; @@ -4406,7 +4406,7 @@ public class TestFromClientSide { public void testAppend() throws Exception { LOG.info("Starting testAppend"); final byte [] TABLENAME = Bytes.toBytes("testAppend"); - HTable t = TEST_UTIL.createTable(TABLENAME, FAMILY); + Table t = TEST_UTIL.createTable(TABLENAME, FAMILY); byte[] v1 = Bytes.toBytes("42"); byte[] v2 = Bytes.toBytes("23"); byte [][] QUALIFIERS = new byte [][] { @@ -4436,7 +4436,7 @@ public class TestFromClientSide { LOG.info("Starting testIncrementWithDeletes"); final TableName TABLENAME = TableName.valueOf("testIncrementWithDeletes"); - HTable ht = TEST_UTIL.createTable(TABLENAME, FAMILY); + Table ht = TEST_UTIL.createTable(TABLENAME, FAMILY); final byte[] COLUMN = Bytes.toBytes("column"); ht.incrementColumnValue(ROW, FAMILY, COLUMN, 5); @@ -4457,7 +4457,7 @@ public class TestFromClientSide { public void testIncrementingInvalidValue() throws Exception { LOG.info("Starting testIncrementingInvalidValue"); final byte [] TABLENAME = Bytes.toBytes("testIncrementingInvalidValue"); - HTable ht = TEST_UTIL.createTable(TABLENAME, FAMILY); + Table ht = TEST_UTIL.createTable(TABLENAME, FAMILY); final byte[] COLUMN = Bytes.toBytes("column"); Put p = new Put(ROW); // write an integer here (not a Long) @@ -4483,7 +4483,7 @@ public class TestFromClientSide { public void testIncrementInvalidArguments() throws Exception { LOG.info("Starting testIncrementInvalidArguments"); final byte[] TABLENAME = Bytes.toBytes("testIncrementInvalidArguments"); - HTable ht = TEST_UTIL.createTable(TABLENAME, FAMILY); + Table ht = TEST_UTIL.createTable(TABLENAME, FAMILY); final byte[] COLUMN = Bytes.toBytes("column"); try { // try null row @@ -4538,7 +4538,7 @@ public class TestFromClientSide { public void testIncrementOutOfOrder() throws Exception { LOG.info("Starting testIncrementOutOfOrder"); final byte [] TABLENAME = Bytes.toBytes("testIncrementOutOfOrder"); - HTable ht = TEST_UTIL.createTable(TABLENAME, FAMILY); + Table ht = TEST_UTIL.createTable(TABLENAME, FAMILY); byte [][] QUALIFIERS = new byte [][] { Bytes.toBytes("B"), Bytes.toBytes("A"), Bytes.toBytes("C") @@ -4578,7 +4578,7 @@ public class TestFromClientSide { public void testIncrement() throws Exception { LOG.info("Starting testIncrement"); final byte [] TABLENAME = Bytes.toBytes("testIncrement"); - HTable ht = TEST_UTIL.createTable(TABLENAME, FAMILY); + Table ht = TEST_UTIL.createTable(TABLENAME, FAMILY); byte [][] ROWS = new byte [][] { Bytes.toBytes("a"), Bytes.toBytes("b"), Bytes.toBytes("c"), @@ -4656,7 +4656,7 @@ public class TestFromClientSide { conf.set(HConstants.HBASE_CLIENT_IPC_POOL_TYPE, "round-robin"); conf.setInt(HConstants.HBASE_CLIENT_IPC_POOL_SIZE, poolSize); - HTable table = TEST_UTIL.createTable(tableName, new byte[][] { FAMILY }, + Table table = TEST_UTIL.createTable(tableName, new byte[][] { FAMILY }, conf, Integer.MAX_VALUE); final long ts = EnvironmentEdgeManager.currentTime(); @@ -4693,7 +4693,7 @@ public class TestFromClientSide { conf.set(HConstants.HBASE_CLIENT_IPC_POOL_TYPE, "thread-local"); conf.setInt(HConstants.HBASE_CLIENT_IPC_POOL_SIZE, poolSize); - final HTable table = TEST_UTIL.createTable(tableName, + final Table table = TEST_UTIL.createTable(tableName, new byte[][] { FAMILY }, conf, 3); final long ts = EnvironmentEdgeManager.currentTime(); @@ -4771,7 +4771,7 @@ public class TestFromClientSide { final byte [] anotherrow = Bytes.toBytes("anotherrow"); final byte [] value2 = Bytes.toBytes("abcd"); - HTable table = TEST_UTIL.createTable(Bytes.toBytes("testCheckAndPut"), + Table table = TEST_UTIL.createTable(Bytes.toBytes("testCheckAndPut"), new byte [][] {FAMILY}); Put put1 = new Put(ROW); put1.add(FAMILY, QUALIFIER, VALUE); @@ -4813,7 +4813,7 @@ public class TestFromClientSide { final byte [] value3 = Bytes.toBytes("cccc"); final byte [] value4 = Bytes.toBytes("dddd"); - HTable table = TEST_UTIL.createTable(Bytes.toBytes("testCheckAndPutWithCompareOp"), + Table table = TEST_UTIL.createTable(Bytes.toBytes("testCheckAndPutWithCompareOp"), new byte [][] {FAMILY}); Put put2 = new Put(ROW); @@ -4879,7 +4879,7 @@ public class TestFromClientSide { final byte [] value3 = Bytes.toBytes("cccc"); final byte [] value4 = Bytes.toBytes("dddd"); - HTable table = TEST_UTIL.createTable(Bytes.toBytes("testCheckAndDeleteWithCompareOp"), + Table table = TEST_UTIL.createTable(Bytes.toBytes("testCheckAndDeleteWithCompareOp"), new byte [][] {FAMILY}); Put put2 = new Put(ROW); @@ -5176,7 +5176,7 @@ public class TestFromClientSide { byte [] family1 = Bytes.toBytes("f1"); byte [] family2 = Bytes.toBytes("f2"); HTable table = TEST_UTIL.createTable(TABLE, new byte[][] {family1, family2}, 10); - HBaseAdmin admin = new HBaseAdmin(TEST_UTIL.getConfiguration()); + Admin admin = new HBaseAdmin(TEST_UTIL.getConfiguration()); Map regionsMap = table.getRegionLocations(); assertEquals(1, regionsMap.size()); HRegionInfo regionInfo = regionsMap.keySet().iterator().next(); @@ -5272,7 +5272,7 @@ public class TestFromClientSide { @Test public void testJira6912() throws Exception { byte [] TABLE = Bytes.toBytes("testJira6912"); - HTable foo = TEST_UTIL.createTable(TABLE, new byte[][] {FAMILY}, 10); + Table foo = TEST_UTIL.createTable(TABLE, new byte[][] {FAMILY}, 10); List puts = new ArrayList(); for (int i=0;i !=100; i++){ @@ -5297,7 +5297,7 @@ public class TestFromClientSide { @Test public void testScan_NullQualifier() throws IOException { - HTable table = TEST_UTIL.createTable(Bytes.toBytes("testScan_NullQualifier"), FAMILY); + Table table = TEST_UTIL.createTable(Bytes.toBytes("testScan_NullQualifier"), FAMILY); Put put = new Put(ROW); put.add(FAMILY, QUALIFIER, VALUE); table.put(put); @@ -5326,7 +5326,7 @@ public class TestFromClientSide { @Test public void testNegativeTimestamp() throws IOException { - HTable table = TEST_UTIL.createTable(Bytes.toBytes("testNegativeTimestamp"), FAMILY); + Table table = TEST_UTIL.createTable(Bytes.toBytes("testNegativeTimestamp"), FAMILY); try { Put put = new Put(ROW, -1); @@ -5471,7 +5471,7 @@ public class TestFromClientSide { @Test public void testRawScanRespectsVersions() throws Exception { byte[] TABLE = Bytes.toBytes("testRawScan"); - HTable table = TEST_UTIL.createTable(TABLE, new byte[][] { FAMILY }); + Table table = TEST_UTIL.createTable(TABLE, new byte[][] { FAMILY }); byte[] row = Bytes.toBytes("row"); // put the same row 4 times, with different values @@ -5547,7 +5547,7 @@ public class TestFromClientSide { public void testSmallScan() throws Exception { // Test Initialization. byte[] TABLE = Bytes.toBytes("testSmallScan"); - HTable table = TEST_UTIL.createTable(TABLE, FAMILY); + Table table = TEST_UTIL.createTable(TABLE, FAMILY); // Insert one row each region int insertNum = 10; @@ -5583,7 +5583,7 @@ public class TestFromClientSide { @Test public void testSuperSimpleWithReverseScan() throws Exception { byte[] TABLE = Bytes.toBytes("testSuperSimpleWithReverseScan"); - HTable ht = TEST_UTIL.createTable(TABLE, FAMILY); + Table ht = TEST_UTIL.createTable(TABLE, FAMILY); Put put = new Put(Bytes.toBytes("0-b11111-0000000000000000000")); put.add(FAMILY, QUALIFIER, VALUE); ht.put(put); @@ -5629,7 +5629,7 @@ public class TestFromClientSide { @Test public void testFiltersWithReverseScan() throws Exception { byte[] TABLE = Bytes.toBytes("testFiltersWithReverseScan"); - HTable ht = TEST_UTIL.createTable(TABLE, FAMILY); + Table ht = TEST_UTIL.createTable(TABLE, FAMILY); byte[][] ROWS = makeN(ROW, 10); byte[][] QUALIFIERS = { Bytes.toBytes("col0--"), Bytes.toBytes("col1--"), @@ -5669,7 +5669,7 @@ public class TestFromClientSide { @Test public void testKeyOnlyFilterWithReverseScan() throws Exception { byte[] TABLE = Bytes.toBytes("testKeyOnlyFilterWithReverseScan"); - HTable ht = TEST_UTIL.createTable(TABLE, FAMILY); + Table ht = TEST_UTIL.createTable(TABLE, FAMILY); byte[][] ROWS = makeN(ROW, 10); byte[][] QUALIFIERS = { Bytes.toBytes("col0--"), Bytes.toBytes("col1--"), @@ -5710,7 +5710,7 @@ public class TestFromClientSide { @Test public void testSimpleMissingWithReverseScan() throws Exception { byte[] TABLE = Bytes.toBytes("testSimpleMissingWithReverseScan"); - HTable ht = TEST_UTIL.createTable(TABLE, FAMILY); + Table ht = TEST_UTIL.createTable(TABLE, FAMILY); byte[][] ROWS = makeN(ROW, 4); // Try to get a row on an empty table @@ -5775,7 +5775,7 @@ public class TestFromClientSide { @Test public void testNullWithReverseScan() throws Exception { byte[] TABLE = Bytes.toBytes("testNullWithReverseScan"); - HTable ht = TEST_UTIL.createTable(TABLE, FAMILY); + Table ht = TEST_UTIL.createTable(TABLE, FAMILY); // Null qualifier (should work) Put put = new Put(ROW); put.add(FAMILY, null, VALUE); @@ -5816,7 +5816,7 @@ public class TestFromClientSide { byte[][] FAMILIES = makeNAscii(FAMILY, 3); byte[][] VALUES = makeN(VALUE, 5); long[] ts = { 1000, 2000, 3000, 4000, 5000 }; - HTable ht = TEST_UTIL.createTable(TABLE, FAMILIES, + Table ht = TEST_UTIL.createTable(TABLE, FAMILIES, TEST_UTIL.getConfiguration(), 3); Put put = new Put(ROW); @@ -6091,7 +6091,7 @@ public class TestFromClientSide { table.close(); } - private void reverseScanTest(HTable table, boolean small) throws IOException { + private void reverseScanTest(Table table, boolean small) throws IOException { // scan backward Scan scan = new Scan(); scan.setReversed(true); diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestFromClientSide3.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestFromClientSide3.java index 64ced173274..5858e75fe2e 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestFromClientSide3.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestFromClientSide3.java @@ -98,7 +98,7 @@ public class TestFromClientSide3 { // Nothing to do. } - private void randomCFPuts(HTable table, byte[] row, byte[] family, int nPuts) + private void randomCFPuts(Table table, byte[] row, byte[] family, int nPuts) throws Exception { Put put = new Put(row); for (int i = 0; i < nPuts; i++) { @@ -269,7 +269,7 @@ public class TestFromClientSide3 { @Test public void testHTableBatchWithEmptyPut() throws Exception { - HTable table = TEST_UTIL.createTable( + Table table = TEST_UTIL.createTable( Bytes.toBytes("testHTableBatchWithEmptyPut"), new byte[][] { FAMILY }); try { List actions = (List) new ArrayList(); @@ -296,7 +296,7 @@ public class TestFromClientSide3 { // Test with a single region table. - HTable table = TEST_UTIL.createTable( + Table table = TEST_UTIL.createTable( Bytes.toBytes("testHTableExistsMethodSingleRegionSingleGet"), new byte[][] { FAMILY }); Put put = new Put(ROW); @@ -336,7 +336,7 @@ public class TestFromClientSide3 { @Test public void testHTableExistsMethodMultipleRegionsSingleGet() throws Exception { - HTable table = TEST_UTIL.createTable( + Table table = TEST_UTIL.createTable( Bytes.toBytes("testHTableExistsMethodMultipleRegionsSingleGet"), new byte[][] { FAMILY }, 1, new byte[] { 0x00 }, new byte[] { (byte) 0xff }, 255); Put put = new Put(ROW); @@ -409,7 +409,7 @@ public class TestFromClientSide3 { HTableDescriptor desc = new HTableDescriptor(TableName.valueOf(Bytes.toBytes("test"))); desc.addFamily(new HColumnDescriptor(FAMILY)); admin.createTable(desc); - HTable table = new HTable(TEST_UTIL.getConfiguration(), "test"); + Table table = new HTable(TEST_UTIL.getConfiguration(), "test"); Put put = new Put(ROW_BYTES); put.add(FAMILY, COL_QUAL, VAL_BYTES); diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestHBaseAdminNoCluster.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestHBaseAdminNoCluster.java index ea92bd526f6..82819daa7ec 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestHBaseAdminNoCluster.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestHBaseAdminNoCluster.java @@ -36,7 +36,6 @@ import org.junit.Test; import org.junit.experimental.categories.Category; import org.mockito.Mockito; import org.mortbay.log.Log; -import org.apache.hadoop.hbase.client.HConnectionTestingUtility; import com.google.protobuf.RpcController; import com.google.protobuf.ServiceException; @@ -71,7 +70,7 @@ public class TestHBaseAdminNoCluster { thenThrow(new ServiceException("Test fail").initCause(new PleaseHoldException("test"))); Mockito.when(connection.getKeepAliveMasterService()).thenReturn(masterAdmin); // Mock up our admin Interfaces - HBaseAdmin admin = new HBaseAdmin(configuration); + Admin admin = new HBaseAdmin(configuration); try { HTableDescriptor htd = new HTableDescriptor(TableName.valueOf("testMasterMonitorCollableRetries")); diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestHCM.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestHCM.java index 0b84ec941d1..7a818aa6cec 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestHCM.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestHCM.java @@ -212,7 +212,7 @@ public class TestHCM { @Test public void testAdminFactory() throws IOException { HConnection con1 = HConnectionManager.createConnection(TEST_UTIL.getConfiguration()); - HBaseAdmin admin = (HBaseAdmin)con1.getAdmin(); + Admin admin = con1.getAdmin(); assertTrue(admin.getConnection() == con1); assertTrue(admin.getConfiguration() == TEST_UTIL.getConfiguration()); con1.close(); @@ -425,7 +425,7 @@ public class TestHCM { c2.setInt(HConstants.HBASE_CLIENT_RETRIES_NUMBER, 1); // Don't retry: retry = test failed c2.setInt(RpcClient.IDLE_TIME, idleTime); - final HTable table = new HTable(c2, tableName.getBytes()); + final Table table = new HTable(c2, tableName.getBytes()); Put put = new Put(ROW); put.add(FAM_NAM, ROW, ROW); @@ -777,9 +777,9 @@ public class TestHCM { */ @Test public void testConnectionManagement() throws Exception{ - HTable table0 = TEST_UTIL.createTable(TABLE_NAME1, FAM_NAM); + Table table0 = TEST_UTIL.createTable(TABLE_NAME1, FAM_NAM); HConnection conn = HConnectionManager.createConnection(TEST_UTIL.getConfiguration()); - HTableInterface table = conn.getTable(TABLE_NAME1.getName()); + Table table = conn.getTable(TABLE_NAME1.getName()); table.close(); assertFalse(conn.isClosed()); assertFalse(((HTable)table).getPool().isShutdown()); @@ -1233,7 +1233,7 @@ public class TestHCM { try { c1 = ConnectionManager.getConnectionInternal(config); LOG.info("HTable connection " + i + " " + c1); - HTable table = new HTable(config, TABLE_NAME4, pool); + Table table = new HTable(config, TABLE_NAME4, pool); table.close(); LOG.info("HTable connection " + i + " closed " + c1); } catch (Exception e) { @@ -1273,7 +1273,7 @@ public class TestHCM { TEST_UTIL.createTable(tableName.getName(), new byte[][] {FAM_NAM}, config).close(); HConnection connection = HConnectionManager.createConnection(config); - HTableInterface table = connection.getTable(tableName); + Table table = connection.getTable(tableName); // this will cache the meta location and table's region location table.get(new Get(Bytes.toBytes("foo"))); diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestHTablePool.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestHTablePool.java index 322ebaa736d..cf6ea921f37 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestHTablePool.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestHTablePool.java @@ -21,6 +21,10 @@ package org.apache.hadoop.hbase.client; import java.io.IOException; 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.PoolMap.PoolType; import org.junit.*; @@ -60,14 +64,14 @@ public class TestHTablePool { String tableName = Bytes.toString(TABLENAME); // Request a table from an empty pool - HTableInterface table = pool.getTable(tableName); + Table table = pool.getTable(tableName); Assert.assertNotNull(table); // Close table (returns table to the pool) table.close(); // Request a table of the same name - HTableInterface sameTable = pool.getTable(tableName); + Table sameTable = pool.getTable(tableName); Assert.assertSame( ((HTablePool.PooledHTable) table).getWrappedTable(), ((HTablePool.PooledHTable) sameTable).getWrappedTable()); @@ -79,14 +83,14 @@ public class TestHTablePool { Integer.MAX_VALUE, getPoolType()); // Request a table from an empty pool - HTableInterface table = pool.getTable(TABLENAME); + Table table = pool.getTable(TABLENAME); Assert.assertNotNull(table); // Close table (returns table to the pool) table.close(); // Request a table of the same name - HTableInterface sameTable = pool.getTable(TABLENAME); + Table sameTable = pool.getTable(TABLENAME); Assert.assertSame( ((HTablePool.PooledHTable) table).getWrappedTable(), ((HTablePool.PooledHTable) sameTable).getWrappedTable()); @@ -104,8 +108,8 @@ public class TestHTablePool { TEST_UTIL.createTable(otherTable, HConstants.CATALOG_FAMILY); // Request a table from an empty pool - HTableInterface table1 = pool.getTable(TABLENAME); - HTableInterface table2 = pool.getTable(otherTable); + Table table1 = pool.getTable(TABLENAME); + Table table2 = pool.getTable(otherTable); Assert.assertNotNull(table2); // Close tables (returns tables to the pool) @@ -113,8 +117,8 @@ public class TestHTablePool { table2.close(); // Request tables of the same names - HTableInterface sameTable1 = pool.getTable(TABLENAME); - HTableInterface sameTable2 = pool.getTable(otherTable); + Table sameTable1 = pool.getTable(TABLENAME); + Table sameTable2 = pool.getTable(otherTable); Assert.assertSame( ((HTablePool.PooledHTable) table1).getWrappedTable(), ((HTablePool.PooledHTable) sameTable1).getWrappedTable()); @@ -129,7 +133,7 @@ public class TestHTablePool { String tableName = Bytes.toString(TABLENAME);// Request a table from // an // empty pool - HTableInterface table = pool.getTable(tableName); + Table table = pool.getTable(tableName); // Test if proxy implementation is returned Assert.assertTrue(table instanceof HTablePool.PooledHTable); @@ -150,7 +154,7 @@ public class TestHTablePool { pool.putTable(table); // Request a table of the same name - HTableInterface sameTable = pool.getTable(tableName); + Table sameTable = pool.getTable(tableName); // test no proxy over proxy created Assert.assertSame(((HTablePool.PooledHTable) table).getWrappedTable(), @@ -166,7 +170,7 @@ public class TestHTablePool { // empty pool // 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(), TABLENAME) { // implementation doesn't matter as long the table is not from @@ -188,7 +192,7 @@ public class TestHTablePool { String tableName = Bytes.toString(TABLENAME); // Request a table from an empty pool - HTableInterface table = pool.getTable(tableName); + Table table = pool.getTable(tableName); Assert.assertNotNull(table); Assert.assertTrue(((HTablePool.PooledHTable) table).isOpen()); // Close table (returns table to the pool) @@ -223,9 +227,9 @@ public class TestHTablePool { getPoolType()); // Request tables from an empty pool - HTableInterface table1 = pool.getTable(TABLENAME); - HTableInterface table2 = pool.getTable(TABLENAME); - HTableInterface table3 = pool.getTable(TABLENAME); + Table table1 = pool.getTable(TABLENAME); + Table table2 = pool.getTable(TABLENAME); + Table table3 = pool.getTable(TABLENAME); // Close tables (returns tables to the pool) table1.close(); @@ -234,9 +238,9 @@ public class TestHTablePool { table3.close(); // Request tables of the same name - HTableInterface sameTable1 = pool.getTable(TABLENAME); - HTableInterface sameTable2 = pool.getTable(TABLENAME); - HTableInterface sameTable3 = pool.getTable(TABLENAME); + Table sameTable1 = pool.getTable(TABLENAME); + Table sameTable2 = pool.getTable(TABLENAME); + Table sameTable3 = pool.getTable(TABLENAME); Assert.assertSame( ((HTablePool.PooledHTable) table1).getWrappedTable(), ((HTablePool.PooledHTable) sameTable1).getWrappedTable()); @@ -264,7 +268,7 @@ public class TestHTablePool { admin.createTable(tableDescriptor); // Request tables from an empty pool - HTableInterface[] tables = new HTableInterface[4]; + Table[] tables = new Table[4]; for (int i = 0; i < 4; ++i) { tables[i] = pool.getTable(TABLENAME); } @@ -298,9 +302,9 @@ public class TestHTablePool { getPoolType()); // Request tables from an empty pool - HTableInterface table1 = pool.getTable(TABLENAME); - HTableInterface table2 = pool.getTable(TABLENAME); - HTableInterface table3 = pool.getTable(TABLENAME); + Table table1 = pool.getTable(TABLENAME); + Table table2 = pool.getTable(TABLENAME); + Table table3 = pool.getTable(TABLENAME); // Close tables (returns tables to the pool) table1.close(); @@ -310,9 +314,9 @@ public class TestHTablePool { table3.close(); // Request tables of the same name - HTableInterface sameTable1 = pool.getTable(TABLENAME); - HTableInterface sameTable2 = pool.getTable(TABLENAME); - HTableInterface sameTable3 = pool.getTable(TABLENAME); + Table sameTable1 = pool.getTable(TABLENAME); + Table sameTable2 = pool.getTable(TABLENAME); + Table sameTable3 = pool.getTable(TABLENAME); Assert.assertSame( ((HTablePool.PooledHTable) table3).getWrappedTable(), ((HTablePool.PooledHTable) sameTable1).getWrappedTable()); @@ -340,7 +344,7 @@ public class TestHTablePool { admin.createTable(tableDescriptor); // Request tables from an empty pool - HTableInterface[] tables = new HTableInterface[4]; + Table[] tables = new Table[4]; for (int i = 0; i < 4; ++i) { tables[i] = pool.getTable(TABLENAME); } diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestMultiParallel.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestMultiParallel.java index 9fc05656248..6e2ff13e929 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestMultiParallel.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestMultiParallel.java @@ -147,7 +147,7 @@ public class TestMultiParallel { */ @Test(timeout=300000) public void testActiveThreadsCount() throws Exception{ - HTable table = new HTable(UTIL.getConfiguration(), TEST_TABLE); + Table table = new HTable(UTIL.getConfiguration(), TEST_TABLE); List puts = constructPutRequests(); // creates a Put for every region table.batch(puts); Field poolField = table.getClass().getDeclaredField("pool"); @@ -160,7 +160,7 @@ public class TestMultiParallel { @Test(timeout=300000) public void testBatchWithGet() throws Exception { LOG.info("test=testBatchWithGet"); - HTable table = new HTable(UTIL.getConfiguration(), TEST_TABLE); + Table table = new HTable(UTIL.getConfiguration(), TEST_TABLE); // load test data List puts = constructPutRequests(); @@ -199,7 +199,7 @@ public class TestMultiParallel { @Test public void testBadFam() throws Exception { LOG.info("test=testBadFam"); - HTable table = new HTable(UTIL.getConfiguration(), TEST_TABLE); + Table table = new HTable(UTIL.getConfiguration(), TEST_TABLE); List actions = new ArrayList(); Put p = new Put(Bytes.toBytes("row1")); @@ -321,7 +321,7 @@ public class TestMultiParallel { @Test (timeout=300000) public void testBatchWithPut() throws Exception { LOG.info("test=testBatchWithPut"); - HTable table = new HTable(UTIL.getConfiguration(), TEST_TABLE); + Table table = new HTable(UTIL.getConfiguration(), TEST_TABLE); // put multiple rows using a batch List puts = constructPutRequests(); @@ -354,7 +354,7 @@ public class TestMultiParallel { @Test(timeout=300000) public void testBatchWithDelete() throws Exception { LOG.info("test=testBatchWithDelete"); - HTable table = new HTable(UTIL.getConfiguration(), TEST_TABLE); + Table table = new HTable(UTIL.getConfiguration(), TEST_TABLE); // Load some data List puts = constructPutRequests(); @@ -383,7 +383,7 @@ public class TestMultiParallel { @Test(timeout=300000) public void testHTableDeleteWithList() throws Exception { LOG.info("test=testHTableDeleteWithList"); - HTable table = new HTable(UTIL.getConfiguration(), TEST_TABLE); + Table table = new HTable(UTIL.getConfiguration(), TEST_TABLE); // Load some data List puts = constructPutRequests(); @@ -412,7 +412,7 @@ public class TestMultiParallel { @Test(timeout=300000) public void testBatchWithManyColsInOneRowGetAndPut() throws Exception { LOG.info("test=testBatchWithManyColsInOneRowGetAndPut"); - HTable table = new HTable(UTIL.getConfiguration(), TEST_TABLE); + Table table = new HTable(UTIL.getConfiguration(), TEST_TABLE); List puts = new ArrayList(); for (int i = 0; i < 100; i++) { @@ -453,7 +453,7 @@ public class TestMultiParallel { final byte[] QUAL2 = Bytes.toBytes("qual2"); final byte[] QUAL3 = Bytes.toBytes("qual3"); final byte[] QUAL4 = Bytes.toBytes("qual4"); - HTable table = new HTable(UTIL.getConfiguration(), TEST_TABLE); + Table table = new HTable(UTIL.getConfiguration(), TEST_TABLE); Delete d = new Delete(ONE_ROW); table.delete(d); Put put = new Put(ONE_ROW); @@ -531,7 +531,7 @@ public class TestMultiParallel { Runnable r = new Runnable() { @Override public void run() { - HTable table = null; + Table table = null; try { table = new HTable(UTIL.getConfiguration(), TEST_TABLE); } catch (IOException e) { @@ -573,7 +573,7 @@ public class TestMultiParallel { @Test(timeout=300000) public void testBatchWithMixedActions() throws Exception { LOG.info("test=testBatchWithMixedActions"); - HTable table = new HTable(UTIL.getConfiguration(), TEST_TABLE); + Table table = new HTable(UTIL.getConfiguration(), TEST_TABLE); // Load some data to start Object[] results = table.batch(constructPutRequests()); @@ -666,7 +666,7 @@ public class TestMultiParallel { return puts; } - private void validateLoadedData(HTable table) throws IOException { + private void validateLoadedData(Table table) throws IOException { // get the data back and validate that it is correct for (byte[] k : KEYS) { Get get = new Get(k); diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestMultipleTimestamps.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestMultipleTimestamps.java index 69cad8a8c78..a28b8286719 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestMultipleTimestamps.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestMultipleTimestamps.java @@ -87,7 +87,7 @@ public class TestMultipleTimestamps { byte [][] FAMILIES = new byte[][] { FAMILY }; // create table; set versions to max... - HTable ht = TEST_UTIL.createTable(TABLE, FAMILIES, Integer.MAX_VALUE); + Table ht = TEST_UTIL.createTable(TABLE, FAMILIES, Integer.MAX_VALUE); Integer[] putRows = new Integer[] {1, 3, 5, 7}; Integer[] putColumns = new Integer[] { 1, 3, 5}; @@ -129,7 +129,7 @@ public class TestMultipleTimestamps { byte [][] FAMILIES = new byte[][] { FAMILY }; // create table; set versions to max... - HTable ht = TEST_UTIL.createTable(TABLE, FAMILIES, Integer.MAX_VALUE); + Table ht = TEST_UTIL.createTable(TABLE, FAMILIES, Integer.MAX_VALUE); Integer[] putRows = new Integer[] {1, 3, 5, 7}; Integer[] putColumns = new Integer[] { 1, 3, 5}; @@ -170,7 +170,7 @@ public class TestMultipleTimestamps { byte [][] FAMILIES = new byte[][] { FAMILY }; // create table; set versions to max... - HTable ht = TEST_UTIL.createTable(TABLE, FAMILIES, Integer.MAX_VALUE); + Table ht = TEST_UTIL.createTable(TABLE, FAMILIES, Integer.MAX_VALUE); Integer[] putRows = new Integer[] {1, 3, 5, 7}; Integer[] putColumns = new Integer[] { 1, 3, 5}; @@ -223,7 +223,7 @@ public class TestMultipleTimestamps { byte [][] FAMILIES = new byte[][] { FAMILY }; // create table; set versions to max... - HTable ht = TEST_UTIL.createTable(TABLE, FAMILIES, Integer.MAX_VALUE); + Table ht = TEST_UTIL.createTable(TABLE, FAMILIES, Integer.MAX_VALUE); Integer[] putRows1 = new Integer[] {1, 2, 3}; Integer[] putColumns1 = new Integer[] { 2, 5, 6}; @@ -293,7 +293,7 @@ public class TestMultipleTimestamps { byte [][] FAMILIES = new byte[][] { FAMILY }; // create table; set versions to max... - HTable ht = TEST_UTIL.createTable(TABLE, FAMILIES, Integer.MAX_VALUE); + Table ht = TEST_UTIL.createTable(TABLE, FAMILIES, Integer.MAX_VALUE); // For row:0, col:0: insert versions 1 through 5. putNVersions(ht, FAMILY, 0, 0, 1, 5); @@ -327,7 +327,7 @@ public class TestMultipleTimestamps { byte [][] FAMILIES = new byte[][] { FAMILY }; // create table; set versions to max... - HTable ht = TEST_UTIL.createTable(TABLE, FAMILIES, Integer.MAX_VALUE); + Table ht = TEST_UTIL.createTable(TABLE, FAMILIES, Integer.MAX_VALUE); // For row:0, col:0: insert versions 1 through 5. putNVersions(ht, FAMILY, 0, 0, 1, 5); @@ -353,7 +353,7 @@ public class TestMultipleTimestamps { byte [][] FAMILIES = new byte[][] { FAMILY }; // create table; set versions to max... - HTable ht = TEST_UTIL.createTable(TABLE, FAMILIES, Integer.MAX_VALUE); + Table ht = TEST_UTIL.createTable(TABLE, FAMILIES, Integer.MAX_VALUE); // For row:0, col:0: insert versions 1 through 5. putNVersions(ht, FAMILY, 0, 0, 1, 5); @@ -379,7 +379,7 @@ public class TestMultipleTimestamps { byte [][] FAMILIES = new byte[][] { FAMILY }; // create table; set versions to max... - HTable ht = TEST_UTIL.createTable(TABLE, FAMILIES, Integer.MAX_VALUE); + Table ht = TEST_UTIL.createTable(TABLE, FAMILIES, Integer.MAX_VALUE); // For row:0, col:0: insert versions 1 through 5. putNVersions(ht, FAMILY, 0, 0, 1, 5); @@ -428,7 +428,7 @@ public class TestMultipleTimestamps { * versions for the row/column specified by rowIdx & colIdx. * */ - private Cell[] getNVersions(HTable ht, byte[] cf, int rowIdx, + private Cell[] getNVersions(Table ht, byte[] cf, int rowIdx, int colIdx, List versions) throws IOException { byte row[] = Bytes.toBytes("row:" + rowIdx); @@ -442,7 +442,7 @@ public class TestMultipleTimestamps { return result.rawCells(); } - private ResultScanner scan(HTable ht, byte[] cf, + private ResultScanner scan(Table ht, byte[] cf, Integer[] rowIndexes, Integer[] columnIndexes, Long[] versions, int maxVersions) throws IOException { @@ -463,7 +463,7 @@ public class TestMultipleTimestamps { return scanner; } - private void put(HTable ht, byte[] cf, Integer[] rowIndexes, + private void put(Table ht, byte[] cf, Integer[] rowIndexes, Integer[] columnIndexes, Long[] versions) throws IOException { for (int rowIdx: rowIndexes) { @@ -485,7 +485,7 @@ public class TestMultipleTimestamps { * Insert in specific row/column versions with timestamps * versionStart..versionEnd. */ - private void putNVersions(HTable ht, byte[] cf, int rowIdx, int colIdx, + private void putNVersions(Table ht, byte[] cf, int rowIdx, int colIdx, long versionStart, long versionEnd) throws IOException { byte row[] = Bytes.toBytes("row:" + rowIdx); @@ -504,7 +504,7 @@ public class TestMultipleTimestamps { * For row/column specified by rowIdx/colIdx, delete the cell * corresponding to the specified version. */ - private void deleteOneVersion(HTable ht, byte[] cf, int rowIdx, + private void deleteOneVersion(Table ht, byte[] cf, int rowIdx, int colIdx, long version) throws IOException { byte row[] = Bytes.toBytes("row:" + rowIdx); @@ -518,7 +518,7 @@ public class TestMultipleTimestamps { * For row/column specified by rowIdx/colIdx, delete all cells * preceeding the specified version. */ - private void deleteAllVersionsBefore(HTable ht, byte[] cf, int rowIdx, + private void deleteAllVersionsBefore(Table ht, byte[] cf, int rowIdx, int colIdx, long version) throws IOException { byte row[] = Bytes.toBytes("row:" + rowIdx); @@ -528,7 +528,7 @@ public class TestMultipleTimestamps { ht.delete(del); } - private void deleteColumn(HTable ht, byte[] cf, int rowIdx, int colIdx) throws IOException { + private void deleteColumn(Table ht, byte[] cf, int rowIdx, int colIdx) throws IOException { byte row[] = Bytes.toBytes("row:" + rowIdx); byte column[] = Bytes.toBytes("column:" + colIdx); Delete del = new Delete(row); @@ -536,7 +536,7 @@ public class TestMultipleTimestamps { ht.delete(del); } - private void deleteFamily(HTable ht, byte[] cf, int rowIdx) throws IOException { + private void deleteFamily(Table ht, byte[] cf, int rowIdx) throws IOException { byte row[] = Bytes.toBytes("row:" + rowIdx); Delete del = new Delete(row); del.deleteFamily(cf); diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestPutWithDelete.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestPutWithDelete.java index 028a8ba8945..aaf37cbc8c0 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestPutWithDelete.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestPutWithDelete.java @@ -52,7 +52,7 @@ public class TestPutWithDelete { final TableName tableName = TableName.valueOf("TestPutWithDelete"); final byte[] rowKey = Bytes.toBytes("12345"); final byte[] family = Bytes.toBytes("cf"); - HTable table = TEST_UTIL.createTable(tableName, family); + Table table = TEST_UTIL.createTable(tableName, family); TEST_UTIL.waitTableAvailable(tableName.getName(), 5000); try { // put one row diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestReplicaWithCluster.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestReplicaWithCluster.java index 77a81466eeb..20a30ea3e57 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestReplicaWithCluster.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestReplicaWithCluster.java @@ -139,7 +139,7 @@ public class TestReplicaWithCluster { HTableDescriptor hdt = HTU.createTableDescriptor("testCreateDeleteTable"); hdt.setRegionReplication(NB_SERVERS); hdt.addCoprocessor(SlowMeCopro.class.getName()); - HTable table = HTU.createTable(hdt, new byte[][]{f}, HTU.getConfiguration()); + Table table = HTU.createTable(hdt, new byte[][]{f}, HTU.getConfiguration()); Put p = new Put(row); p.add(f, row, row); @@ -171,7 +171,7 @@ public class TestReplicaWithCluster { HTableDescriptor hdt = HTU.createTableDescriptor("testChangeTable"); hdt.setRegionReplication(NB_SERVERS); hdt.addCoprocessor(SlowMeCopro.class.getName()); - HTable table = HTU.createTable(hdt, new byte[][]{f}, HTU.getConfiguration()); + Table table = HTU.createTable(hdt, new byte[][]{f}, HTU.getConfiguration()); // basic test: it should work. Put p = new Put(row); @@ -213,7 +213,7 @@ public class TestReplicaWithCluster { } HTU.getHBaseCluster().stopMaster(0); - HBaseAdmin admin = new HBaseAdmin(HTU.getConfiguration()); + Admin admin = new HBaseAdmin(HTU.getConfiguration()); nHdt =admin.getTableDescriptor(hdt.getTableName()); Assert.assertEquals("fams=" + Arrays.toString(nHdt.getColumnFamilies()), bHdt.getColumnFamilies().length + 1, nHdt.getColumnFamilies().length); @@ -253,7 +253,7 @@ public class TestReplicaWithCluster { Put p = new Put(row); p.add(row, row, row); - final HTable table = new HTable(HTU.getConfiguration(), hdt.getTableName()); + final Table table = new HTable(HTU.getConfiguration(), hdt.getTableName()); table.put(p); HTU.getHBaseAdmin().flush(table.getName()); @@ -277,7 +277,7 @@ public class TestReplicaWithCluster { table.close(); LOG.info("stale get on the first cluster done. Now for the second."); - final HTable table2 = new HTable(HTU.getConfiguration(), hdt.getTableName()); + final Table table2 = new HTable(HTU.getConfiguration(), hdt.getTableName()); Waiter.waitFor(HTU.getConfiguration(), 1000, new Waiter.Predicate() { @Override public boolean evaluate() throws Exception { @@ -312,7 +312,7 @@ public class TestReplicaWithCluster { HTableDescriptor hdt = HTU.createTableDescriptor("testBulkLoad"); hdt.setRegionReplication(NB_SERVERS); hdt.addCoprocessor(SlowMeCopro.class.getName()); - HTable table = HTU.createTable(hdt, new byte[][]{f}, HTU.getConfiguration()); + Table table = HTU.createTable(hdt, new byte[][]{f}, HTU.getConfiguration()); // create hfiles to load. LOG.debug("Creating test data"); diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestReplicasClient.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestReplicasClient.java index 1f1f3d4cccc..3fa6c95cca1 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestReplicasClient.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestReplicasClient.java @@ -174,7 +174,7 @@ public class TestReplicasClient { TestRegionServerNoMaster.stopMasterAndAssignMeta(HTU); Configuration c = new Configuration(HTU.getConfiguration()); c.setInt(HConstants.HBASE_CLIENT_RETRIES_NUMBER, 1); - HBaseAdmin ha = new HBaseAdmin(c); + Admin ha = new HBaseAdmin(c); for (boolean masterRuns = true; masterRuns; ) { Thread.sleep(100); try { diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestRpcControllerFactory.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestRpcControllerFactory.java index 02c2ef8f17f..414c4c8549d 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestRpcControllerFactory.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestRpcControllerFactory.java @@ -130,7 +130,7 @@ public class TestRpcControllerFactory { // change one of the connection properties so we get a new HConnection with our configuration conf.setInt(HConstants.HBASE_RPC_TIMEOUT_KEY, HConstants.DEFAULT_HBASE_RPC_TIMEOUT + 1); - HTable table = new HTable(conf, name); + Table table = new HTable(conf, name); table.setAutoFlushTo(false); byte[] row = Bytes.toBytes("row"); Put p = new Put(row); @@ -188,7 +188,7 @@ public class TestRpcControllerFactory { table.close(); } - int doScan(HTable table, Scan scan, int expectedCount) throws IOException { + int doScan(Table table, Scan scan, int expectedCount) throws IOException { ResultScanner results = table.getScanner(scan); results.next(); results.close(); diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestScannerTimeout.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestScannerTimeout.java index 65fb0d6ae71..402f7980653 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestScannerTimeout.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestScannerTimeout.java @@ -65,7 +65,7 @@ public class TestScannerTimeout { c.setInt(HConstants.THREAD_WAKE_FREQUENCY, THREAD_WAKE_FREQUENCY); // We need more than one region server for this test TEST_UTIL.startMiniCluster(2); - HTable table = TEST_UTIL.createTable(TABLE_NAME, SOME_BYTES); + Table table = TEST_UTIL.createTable(TABLE_NAME, SOME_BYTES); for (int i = 0; i < NB_ROWS; i++) { Put put = new Put(Bytes.toBytes(i)); put.add(SOME_BYTES, SOME_BYTES, SOME_BYTES); @@ -99,7 +99,7 @@ public class TestScannerTimeout { LOG.info("START ************ test2481"); Scan scan = new Scan(); scan.setCaching(1); - HTable table = + Table table = new HTable(new Configuration(TEST_UTIL.getConfiguration()), TABLE_NAME); ResultScanner r = table.getScanner(scan); int count = 0; @@ -139,7 +139,7 @@ public class TestScannerTimeout { // this new table Configuration conf = new Configuration(TEST_UTIL.getConfiguration()); conf.setInt(HConstants.HBASE_CLIENT_SCANNER_TIMEOUT_PERIOD, SCANNER_TIMEOUT * 100); - HTable higherScanTimeoutTable = new HTable(conf, TABLE_NAME); + Table higherScanTimeoutTable = new HTable(conf, TABLE_NAME); ResultScanner r = higherScanTimeoutTable.getScanner(scan); // This takes way less than SCANNER_TIMEOUT*100 rs.abort("die!"); @@ -173,7 +173,7 @@ public class TestScannerTimeout { Configuration conf = new Configuration(TEST_UTIL.getConfiguration()); conf.setInt( HConstants.HBASE_CLIENT_SCANNER_TIMEOUT_PERIOD, SCANNER_TIMEOUT*100); - HTable table = new HTable(conf, TABLE_NAME); + Table table = new HTable(conf, TABLE_NAME); LOG.info("START ************ TEST3686A---22"); ResultScanner r = table.getScanner(scan); @@ -212,7 +212,7 @@ public class TestScannerTimeout { // this new table Configuration conf = new Configuration(TEST_UTIL.getConfiguration()); conf.setInt(HConstants.HBASE_CLIENT_SCANNER_TIMEOUT_PERIOD, SCANNER_TIMEOUT * 100); - HTable higherScanTimeoutTable = new HTable(conf, TABLE_NAME); + Table higherScanTimeoutTable = new HTable(conf, TABLE_NAME); ResultScanner r = higherScanTimeoutTable.getScanner(scan); int count = 1; r.next(); diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestScannersFromClientSide.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestScannersFromClientSide.java index 87232192a71..ee5b4ed37b8 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestScannersFromClientSide.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestScannersFromClientSide.java @@ -106,7 +106,7 @@ public class TestScannersFromClientSide { byte [] TABLE = Bytes.toBytes("testScanBatch"); byte [][] QUALIFIERS = HTestConst.makeNAscii(QUALIFIER, 8); - HTable ht = TEST_UTIL.createTable(TABLE, FAMILY); + Table ht = TEST_UTIL.createTable(TABLE, FAMILY); Put put; Scan scan; @@ -182,7 +182,7 @@ public class TestScannersFromClientSide { byte [][] FAMILIES = HTestConst.makeNAscii(FAMILY, 3); byte [][] QUALIFIERS = HTestConst.makeNAscii(QUALIFIER, 20); - HTable ht = TEST_UTIL.createTable(TABLE, FAMILIES); + Table ht = TEST_UTIL.createTable(TABLE, FAMILIES); Get get; Put put; @@ -303,7 +303,7 @@ public class TestScannersFromClientSide { byte [][] FAMILIES = HTestConst.makeNAscii(FAMILY, 3); byte [][] QUALIFIERS = HTestConst.makeNAscii(QUALIFIER, 10); - HTable ht = TEST_UTIL.createTable(TABLE, FAMILIES); + Table ht = TEST_UTIL.createTable(TABLE, FAMILIES); Put put; Scan scan; @@ -352,7 +352,7 @@ public class TestScannersFromClientSide { byte [][] FAMILIES = HTestConst.makeNAscii(FAMILY, 3); byte [][] QUALIFIERS = HTestConst.makeNAscii(QUALIFIER, 20); - HTable ht = TEST_UTIL.createTable(TABLE, FAMILIES); + Table ht = TEST_UTIL.createTable(TABLE, FAMILIES); Get get; Put put; diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestSnapshotCloneIndependence.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestSnapshotCloneIndependence.java index d0da0b2227c..5f2d2c56c54 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestSnapshotCloneIndependence.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestSnapshotCloneIndependence.java @@ -212,7 +212,7 @@ public class TestSnapshotCloneIndependence { TableName cloneTableName = TableName.valueOf("test-clone-" + localTableName); admin.cloneSnapshot(snapshotName, cloneTableName); - HTable clonedTable = new HTable(UTIL.getConfiguration(), cloneTableName); + Table clonedTable = new HTable(UTIL.getConfiguration(), cloneTableName); try { final int clonedTableRowCount = UTIL.countRows(clonedTable); diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestSnapshotMetadata.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestSnapshotMetadata.java index f441d59ac32..f4176494e05 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestSnapshotMetadata.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestSnapshotMetadata.java @@ -169,7 +169,7 @@ public class TestSnapshotMetadata { assertTrue(htd.getConfiguration().size() > 0); admin.createTable(htd); - HTable original = new HTable(UTIL.getConfiguration(), originalTableName); + Table original = new HTable(UTIL.getConfiguration(), originalTableName); originalTableName = TableName.valueOf(sourceTableNameAsString); originalTableDescriptor = admin.getTableDescriptor(originalTableName); originalTableDescription = originalTableDescriptor.toStringCustomizedValues(); @@ -201,7 +201,7 @@ public class TestSnapshotMetadata { familiesList, snapshotNameAsString, rootDir, fs, /* onlineSnapshot= */ false); admin.cloneSnapshot(snapshotName, clonedTableName); - HTable clonedTable = new HTable(UTIL.getConfiguration(), clonedTableName); + Table clonedTable = new HTable(UTIL.getConfiguration(), clonedTableName); HTableDescriptor cloneHtd = admin.getTableDescriptor(clonedTableName); assertEquals( originalTableDescription.replace(originalTableName.getNameAsString(),clonedTableNameAsString), @@ -312,7 +312,7 @@ public class TestSnapshotMetadata { admin.enableTable(originalTableName); // verify that the descrption is reverted - HTable original = new HTable(UTIL.getConfiguration(), originalTableName); + Table original = new HTable(UTIL.getConfiguration(), originalTableName); try { assertTrue(originalTableDescriptor.equals(admin.getTableDescriptor(originalTableName))); assertTrue(originalTableDescriptor.equals(original.getTableDescriptor())); diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestTimestampsFilter.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestTimestampsFilter.java index 05f9c53ef42..3e382ea7235 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestTimestampsFilter.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestTimestampsFilter.java @@ -96,7 +96,7 @@ public class TestTimestampsFilter { Cell kvs[]; // create table; set versions to max... - HTable ht = TEST_UTIL.createTable(TABLE, FAMILIES, Integer.MAX_VALUE); + Table ht = TEST_UTIL.createTable(TABLE, FAMILIES, Integer.MAX_VALUE); for (int rowIdx = 0; rowIdx < 5; rowIdx++) { for (int colIdx = 0; colIdx < 5; colIdx++) { @@ -171,7 +171,7 @@ public class TestTimestampsFilter { byte [][] FAMILIES = new byte[][] { FAMILY }; // create table; set versions to max... - HTable ht = TEST_UTIL.createTable(TABLE, FAMILIES, Integer.MAX_VALUE); + Table ht = TEST_UTIL.createTable(TABLE, FAMILIES, Integer.MAX_VALUE); Put p = new Put(Bytes.toBytes("row")); p.add(FAMILY, Bytes.toBytes("column0"), 3, Bytes.toBytes("value0-3")); @@ -231,7 +231,7 @@ public class TestTimestampsFilter { byte [][] FAMILIES = new byte[][] { FAMILY }; // create table; set versions to max... - HTable ht = TEST_UTIL.createTable(TABLE, FAMILIES, Integer.MAX_VALUE); + Table ht = TEST_UTIL.createTable(TABLE, FAMILIES, Integer.MAX_VALUE); // For row:0, col:0: insert versions 1 through 5. putNVersions(ht, FAMILY, 0, 0, 1, 5); @@ -254,7 +254,7 @@ public class TestTimestampsFilter { ht.close(); } - private void verifyInsertedValues(HTable ht, byte[] cf) throws IOException { + private void verifyInsertedValues(Table ht, byte[] cf) throws IOException { for (int rowIdx = 0; rowIdx < 5; rowIdx++) { for (int colIdx = 0; colIdx < 5; colIdx++) { // ask for versions that exist. @@ -313,7 +313,7 @@ public class TestTimestampsFilter { * versions for the row/column specified by rowIdx & colIdx. * */ - private Cell[] getNVersions(HTable ht, byte[] cf, int rowIdx, + private Cell[] getNVersions(Table ht, byte[] cf, int rowIdx, int colIdx, List versions) throws IOException { byte row[] = Bytes.toBytes("row:" + rowIdx); @@ -332,7 +332,7 @@ public class TestTimestampsFilter { * Uses the TimestampFilter on a Scan to request a specified list of * versions for the rows from startRowIdx to endRowIdx (both inclusive). */ - private Result[] scanNVersions(HTable ht, byte[] cf, int startRowIdx, + private Result[] scanNVersions(Table ht, byte[] cf, int startRowIdx, int endRowIdx, List versions) throws IOException { byte startRow[] = Bytes.toBytes("row:" + startRowIdx); @@ -349,7 +349,7 @@ public class TestTimestampsFilter { * Insert in specific row/column versions with timestamps * versionStart..versionEnd. */ - private void putNVersions(HTable ht, byte[] cf, int rowIdx, int colIdx, + private void putNVersions(Table ht, byte[] cf, int rowIdx, int colIdx, long versionStart, long versionEnd) throws IOException { byte row[] = Bytes.toBytes("row:" + rowIdx); @@ -368,7 +368,7 @@ public class TestTimestampsFilter { * For row/column specified by rowIdx/colIdx, delete the cell * corresponding to the specified version. */ - private void deleteOneVersion(HTable ht, byte[] cf, int rowIdx, + private void deleteOneVersion(Table ht, byte[] cf, int rowIdx, int colIdx, long version) throws IOException { byte row[] = Bytes.toBytes("row:" + rowIdx); diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/constraint/TestConstraint.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/constraint/TestConstraint.java index 919fd246d3c..5b3062542e3 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/constraint/TestConstraint.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/constraint/TestConstraint.java @@ -34,6 +34,7 @@ import org.apache.hadoop.hbase.TableName; import org.apache.hadoop.hbase.client.HTable; import org.apache.hadoop.hbase.client.Put; import org.apache.hadoop.hbase.client.RetriesExhaustedWithDetailsException; +import org.apache.hadoop.hbase.client.Table; import org.apache.hadoop.hbase.coprocessor.CoprocessorHost; import org.apache.hadoop.hbase.util.Bytes; import org.junit.After; @@ -80,7 +81,7 @@ public class TestConstraint { Constraints.add(desc, CheckWasRunConstraint.class); util.getHBaseAdmin().createTable(desc); - HTable table = new HTable(util.getConfiguration(), tableName); + Table table = new HTable(util.getConfiguration(), tableName); try { // test that we don't fail on a valid put Put put = new Put(row1); @@ -112,7 +113,7 @@ public class TestConstraint { Constraints.add(desc, AllFailConstraint.class); util.getHBaseAdmin().createTable(desc); - HTable table = new HTable(util.getConfiguration(), tableName); + Table table = new HTable(util.getConfiguration(), tableName); // test that we do fail on violation Put put = new Put(row1); @@ -155,7 +156,7 @@ public class TestConstraint { Constraints.disableConstraint(desc, AllFailConstraint.class); util.getHBaseAdmin().createTable(desc); - HTable table = new HTable(util.getConfiguration(), tableName); + Table table = new HTable(util.getConfiguration(), tableName); try { // test that we don't fail because its disabled Put put = new Put(row1); @@ -187,7 +188,7 @@ public class TestConstraint { Constraints.disable(desc); util.getHBaseAdmin().createTable(desc); - HTable table = new HTable(util.getConfiguration(), tableName); + Table table = new HTable(util.getConfiguration(), tableName); try { // test that we do fail on violation Put put = new Put(row1); @@ -219,7 +220,7 @@ public class TestConstraint { CheckWasRunConstraint.wasRun = false; util.getHBaseAdmin().createTable(desc); - HTable table = new HTable(util.getConfiguration(), tableName); + Table table = new HTable(util.getConfiguration(), tableName); // test that we do fail on violation Put put = new Put(row1); diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/coprocessor/TestBatchCoprocessorEndpoint.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/coprocessor/TestBatchCoprocessorEndpoint.java index 655d663cbfa..88915250230 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/coprocessor/TestBatchCoprocessorEndpoint.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/coprocessor/TestBatchCoprocessorEndpoint.java @@ -25,6 +25,8 @@ import java.util.Collections; import java.util.Map; import java.util.TreeMap; +import org.apache.hadoop.hbase.client.Admin; +import org.apache.hadoop.hbase.client.Table; import org.apache.hadoop.hbase.util.ByteStringer; import org.apache.commons.logging.Log; import org.apache.commons.logging.LogFactory; @@ -83,14 +85,14 @@ public class TestBatchCoprocessorEndpoint { conf.setStrings(CoprocessorHost.MASTER_COPROCESSOR_CONF_KEY, ProtobufCoprocessorService.class.getName()); util.startMiniCluster(2); - HBaseAdmin admin = new HBaseAdmin(conf); + Admin admin = new HBaseAdmin(conf); HTableDescriptor desc = new HTableDescriptor(TEST_TABLE); desc.addFamily(new HColumnDescriptor(TEST_FAMILY)); admin.createTable(desc, new byte[][]{ROWS[rowSeperator1], ROWS[rowSeperator2]}); util.waitUntilAllRegionsAssigned(TEST_TABLE); admin.close(); - HTable table = new HTable(conf, TEST_TABLE); + Table table = new HTable(conf, TEST_TABLE); for (int i = 0; i < ROWSIZE; i++) { Put put = new Put(ROWS[i]); put.add(TEST_FAMILY, TEST_QUALIFIER, Bytes.toBytes(i)); @@ -106,7 +108,7 @@ public class TestBatchCoprocessorEndpoint { @Test public void testAggregationNullResponse() throws Throwable { - HTable table = new HTable(util.getConfiguration(), TEST_TABLE); + Table table = new HTable(util.getConfiguration(), TEST_TABLE); ColumnAggregationWithNullResponseProtos.SumRequest.Builder builder = ColumnAggregationWithNullResponseProtos.SumRequest .newBuilder(); @@ -143,7 +145,7 @@ public class TestBatchCoprocessorEndpoint { return ret; } - private Map sum(final HTable table, final byte[] family, + private Map sum(final Table table, final byte[] family, final byte[] qualifier, final byte[] start, final byte[] end) throws ServiceException, Throwable { ColumnAggregationProtos.SumRequest.Builder builder = ColumnAggregationProtos.SumRequest @@ -159,7 +161,7 @@ public class TestBatchCoprocessorEndpoint { @Test public void testAggregationWithReturnValue() throws Throwable { - HTable table = new HTable(util.getConfiguration(), TEST_TABLE); + Table table = new HTable(util.getConfiguration(), TEST_TABLE); Map results = sum(table, TEST_FAMILY, TEST_QUALIFIER, ROWS[0], ROWS[ROWS.length - 1]); int sumResult = 0; @@ -195,7 +197,7 @@ public class TestBatchCoprocessorEndpoint { @Test public void testAggregation() throws Throwable { - HTable table = new HTable(util.getConfiguration(), TEST_TABLE); + Table table = new HTable(util.getConfiguration(), TEST_TABLE); Map results = sum(table, TEST_FAMILY, TEST_QUALIFIER, ROWS[0], ROWS[ROWS.length - 1]); int sumResult = 0; @@ -228,7 +230,7 @@ public class TestBatchCoprocessorEndpoint { @Test public void testAggregationWithErrors() throws Throwable { - HTable table = new HTable(util.getConfiguration(), TEST_TABLE); + Table table = new HTable(util.getConfiguration(), TEST_TABLE); final Map results = Collections.synchronizedMap( new TreeMap( diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/coprocessor/TestCoprocessorEndpoint.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/coprocessor/TestCoprocessorEndpoint.java index e67bb9a5570..c8e2c8c326e 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/coprocessor/TestCoprocessorEndpoint.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/coprocessor/TestCoprocessorEndpoint.java @@ -31,6 +31,7 @@ import java.util.NavigableMap; import java.util.TreeMap; import org.apache.hadoop.hbase.client.Admin; +import org.apache.hadoop.hbase.client.Table; import org.apache.hadoop.hbase.util.ByteStringer; import org.apache.commons.logging.Log; import org.apache.commons.logging.LogFactory; @@ -92,14 +93,14 @@ public class TestCoprocessorEndpoint { conf.setStrings(CoprocessorHost.MASTER_COPROCESSOR_CONF_KEY, ProtobufCoprocessorService.class.getName()); util.startMiniCluster(2); - HBaseAdmin admin = new HBaseAdmin(conf); + Admin admin = new HBaseAdmin(conf); HTableDescriptor desc = new HTableDescriptor(TEST_TABLE); desc.addFamily(new HColumnDescriptor(TEST_FAMILY)); admin.createTable(desc, new byte[][]{ROWS[rowSeperator1], ROWS[rowSeperator2]}); util.waitUntilAllRegionsAssigned(TEST_TABLE); admin.close(); - HTable table = new HTable(conf, TEST_TABLE); + Table table = new HTable(conf, TEST_TABLE); for (int i = 0; i < ROWSIZE; i++) { Put put = new Put(ROWS[i]); put.add(TEST_FAMILY, TEST_QUALIFIER, Bytes.toBytes(i)); @@ -113,7 +114,7 @@ public class TestCoprocessorEndpoint { util.shutdownMiniCluster(); } - private Map sum(final HTable table, final byte [] family, + private Map sum(final Table table, final byte [] family, final byte [] qualifier, final byte [] start, final byte [] end) throws ServiceException, Throwable { return table.coprocessorService(ColumnAggregationProtos.ColumnAggregationService.class, @@ -138,7 +139,7 @@ public class TestCoprocessorEndpoint { @Test public void testAggregation() throws Throwable { - HTable table = new HTable(util.getConfiguration(), TEST_TABLE); + Table table = new HTable(util.getConfiguration(), TEST_TABLE); Map results = sum(table, TEST_FAMILY, TEST_QUALIFIER, ROWS[0], ROWS[ROWS.length-1]); int sumResult = 0; @@ -297,7 +298,7 @@ public class TestCoprocessorEndpoint { Configuration configuration = new Configuration(util.getConfiguration()); // Make it not retry forever configuration.setInt(HConstants.HBASE_CLIENT_RETRIES_NUMBER, 1); - HTable table = new HTable(configuration, TEST_TABLE); + Table table = new HTable(configuration, TEST_TABLE); try { CoprocessorRpcChannel protocol = table.coprocessorService(ROWS[0]); diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/coprocessor/TestHTableWrapper.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/coprocessor/TestHTableWrapper.java index b29bec4ce87..cbeb532c9f5 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/coprocessor/TestHTableWrapper.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/coprocessor/TestHTableWrapper.java @@ -33,7 +33,6 @@ import org.apache.hadoop.hbase.client.Append; import org.apache.hadoop.hbase.client.Delete; import org.apache.hadoop.hbase.client.Durability; import org.apache.hadoop.hbase.client.Get; -import org.apache.hadoop.hbase.client.HTable; import org.apache.hadoop.hbase.client.HTableInterface; import org.apache.hadoop.hbase.client.Increment; import org.apache.hadoop.hbase.client.Put; @@ -42,6 +41,7 @@ import org.apache.hadoop.hbase.client.ResultScanner; import org.apache.hadoop.hbase.client.Row; import org.apache.hadoop.hbase.client.RowMutations; import org.apache.hadoop.hbase.client.Scan; +import org.apache.hadoop.hbase.client.Table; import org.apache.hadoop.hbase.client.coprocessor.Batch; import org.apache.hadoop.hbase.ipc.CoprocessorRpcChannel; import org.apache.hadoop.hbase.master.MasterCoprocessorHost; @@ -86,7 +86,7 @@ public class TestHTableWrapper { } private HTableInterface hTableInterface; - private HTable table; + private Table table; @BeforeClass public static void setupBeforeClass() throws Exception { diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/coprocessor/TestOpenTableInCoprocessor.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/coprocessor/TestOpenTableInCoprocessor.java index 13cb9065798..9f8115189c9 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/coprocessor/TestOpenTableInCoprocessor.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/coprocessor/TestOpenTableInCoprocessor.java @@ -35,13 +35,12 @@ import org.apache.hadoop.hbase.MediumTests; import org.apache.hadoop.hbase.TableName; import org.apache.hadoop.hbase.client.Admin; import org.apache.hadoop.hbase.client.Durability; -import org.apache.hadoop.hbase.client.HBaseAdmin; import org.apache.hadoop.hbase.client.HTable; -import org.apache.hadoop.hbase.client.HTableInterface; import org.apache.hadoop.hbase.client.Put; import org.apache.hadoop.hbase.client.Result; import org.apache.hadoop.hbase.client.ResultScanner; import org.apache.hadoop.hbase.client.Scan; +import org.apache.hadoop.hbase.client.Table; import org.apache.hadoop.hbase.regionserver.wal.WALEdit; import org.apache.hadoop.hbase.util.Threads; import org.junit.After; @@ -69,7 +68,7 @@ public class TestOpenTableInCoprocessor { @Override public void prePut(final ObserverContext e, final Put put, final WALEdit edit, final Durability durability) throws IOException { - HTableInterface table = e.getEnvironment().getTable(otherTable); + Table table = e.getEnvironment().getTable(otherTable); table.put(put); table.flushCommits(); completed[0] = true; @@ -102,7 +101,7 @@ public class TestOpenTableInCoprocessor { @Override public void prePut(final ObserverContext e, final Put put, final WALEdit edit, final Durability durability) throws IOException { - HTableInterface table = e.getEnvironment().getTable(otherTable, getPool()); + Table table = e.getEnvironment().getTable(otherTable, getPool()); Put p = new Put(new byte[] { 'a' }); p.add(family, null, new byte[] { 'a' }); try { @@ -162,14 +161,14 @@ public class TestOpenTableInCoprocessor { admin.createTable(primary); admin.createTable(other); - HTable table = new HTable(UTIL.getConfiguration(), "primary"); + Table table = new HTable(UTIL.getConfiguration(), "primary"); Put p = new Put(new byte[] { 'a' }); p.add(family, null, new byte[] { 'a' }); table.put(p); table.flushCommits(); table.close(); - HTable target = new HTable(UTIL.getConfiguration(), otherTable); + Table target = new HTable(UTIL.getConfiguration(), otherTable); assertTrue("Didn't complete update to target table!", completeCheck[0]); assertEquals("Didn't find inserted row", 1, getKeyValueCount(target)); target.close(); @@ -181,7 +180,7 @@ public class TestOpenTableInCoprocessor { * @return number of keyvalues over all rows in the table * @throws IOException */ - private int getKeyValueCount(HTable table) throws IOException { + private int getKeyValueCount(Table table) throws IOException { Scan scan = new Scan(); scan.setMaxVersions(Integer.MAX_VALUE - 1); diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/coprocessor/TestRegionObserverBypass.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/coprocessor/TestRegionObserverBypass.java index 6997ef5b143..bb16add4e0e 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/coprocessor/TestRegionObserverBypass.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/coprocessor/TestRegionObserverBypass.java @@ -29,17 +29,16 @@ import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.hbase.Cell; import org.apache.hadoop.hbase.HBaseConfiguration; import org.apache.hadoop.hbase.HBaseTestingUtility; -import org.apache.hadoop.hbase.KeyValue; import org.apache.hadoop.hbase.MediumTests; import org.apache.hadoop.hbase.TableName; import org.apache.hadoop.hbase.client.Admin; import org.apache.hadoop.hbase.client.Delete; import org.apache.hadoop.hbase.client.Get; -import org.apache.hadoop.hbase.client.HBaseAdmin; import org.apache.hadoop.hbase.client.HTable; import org.apache.hadoop.hbase.client.Put; import org.apache.hadoop.hbase.client.Result; import org.apache.hadoop.hbase.client.Durability; +import org.apache.hadoop.hbase.client.Table; import org.apache.hadoop.hbase.regionserver.wal.WALEdit; import org.apache.hadoop.hbase.util.Bytes; import org.apache.hadoop.hbase.util.EnvironmentEdgeManager; @@ -93,7 +92,7 @@ public class TestRegionObserverBypass { */ @Test public void testSimple() throws Exception { - HTable t = new HTable(util.getConfiguration(), tableName); + Table t = new HTable(util.getConfiguration(), tableName); Put p = new Put(row1); p.add(test,dummy,dummy); // before HBASE-4331, this would throw an exception @@ -112,7 +111,7 @@ public class TestRegionObserverBypass { //previous deletes will eclipse successive puts having the same timestamp EnvironmentEdgeManagerTestHelper.injectEdge(new IncrementingEnvironmentEdge()); - HTable t = new HTable(util.getConfiguration(), tableName); + Table t = new HTable(util.getConfiguration(), tableName); List puts = new ArrayList(); Put p = new Put(row1); p.add(dummy,dummy,dummy); @@ -197,7 +196,7 @@ public class TestRegionObserverBypass { EnvironmentEdgeManager.reset(); } - private void checkRowAndDelete(HTable t, byte[] row, int count) throws IOException { + private void checkRowAndDelete(Table t, byte[] row, int count) throws IOException { Get g = new Get(row); Result r = t.get(g); assertEquals(count, r.size()); diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/coprocessor/TestRegionObserverInterface.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/coprocessor/TestRegionObserverInterface.java index 61c1721058e..d7ec06443b3 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/coprocessor/TestRegionObserverInterface.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/coprocessor/TestRegionObserverInterface.java @@ -58,6 +58,7 @@ import org.apache.hadoop.hbase.client.Result; import org.apache.hadoop.hbase.client.ResultScanner; import org.apache.hadoop.hbase.client.RowMutations; import org.apache.hadoop.hbase.client.Scan; +import org.apache.hadoop.hbase.client.Table; import org.apache.hadoop.hbase.io.hfile.CacheConfig; import org.apache.hadoop.hbase.io.hfile.HFile; import org.apache.hadoop.hbase.io.hfile.HFileContext; @@ -114,7 +115,7 @@ public class TestRegionObserverInterface { TableName tableName = TableName.valueOf(TEST_TABLE.getNameAsString() + ".testRegionObserver"); // recreate table every time in order to reset the status of the // coprocessor. - HTable table = util.createTable(tableName, new byte[][] {A, B, C}); + Table table = util.createTable(tableName, new byte[][] {A, B, C}); try { verifyMethodResult(SimpleRegionObserver.class, new String[] { "hadPreGet", "hadPostGet", "hadPrePut", "hadPostPut", "hadDelete", "hadPostStartRegionOperation", @@ -176,7 +177,7 @@ public class TestRegionObserverInterface { @Test public void testRowMutation() throws IOException { TableName tableName = TableName.valueOf(TEST_TABLE.getNameAsString() + ".testRowMutation"); - HTable table = util.createTable(tableName, new byte[][] {A, B, C}); + Table table = util.createTable(tableName, new byte[][] {A, B, C}); try { verifyMethodResult(SimpleRegionObserver.class, new String[] {"hadPreGet", "hadPostGet", "hadPrePut", "hadPostPut", @@ -213,7 +214,7 @@ public class TestRegionObserverInterface { @Test public void testIncrementHook() throws IOException { TableName tableName = TableName.valueOf(TEST_TABLE.getNameAsString() + ".testIncrementHook"); - HTable table = util.createTable(tableName, new byte[][] {A, B, C}); + Table table = util.createTable(tableName, new byte[][] {A, B, C}); try { Increment inc = new Increment(Bytes.toBytes(0)); inc.addColumn(A, A, 1); @@ -241,7 +242,7 @@ public class TestRegionObserverInterface { public void testCheckAndPutHooks() throws IOException { TableName tableName = TableName.valueOf(TEST_TABLE.getNameAsString() + ".testCheckAndPutHooks"); - HTable table = util.createTable(tableName, new byte[][] {A, B, C}); + Table table = util.createTable(tableName, new byte[][] {A, B, C}); try { Put p = new Put(Bytes.toBytes(0)); p.add(A, A, A); @@ -272,7 +273,7 @@ public class TestRegionObserverInterface { public void testCheckAndDeleteHooks() throws IOException { TableName tableName = TableName.valueOf(TEST_TABLE.getNameAsString() + ".testCheckAndDeleteHooks"); - HTable table = util.createTable(tableName, new byte[][] {A, B, C}); + Table table = util.createTable(tableName, new byte[][] {A, B, C}); try { Put p = new Put(Bytes.toBytes(0)); p.add(A, A, A); @@ -302,7 +303,7 @@ public class TestRegionObserverInterface { @Test public void testAppendHook() throws IOException { TableName tableName = TableName.valueOf(TEST_TABLE.getNameAsString() + ".testAppendHook"); - HTable table = util.createTable(tableName, new byte[][] {A, B, C}); + Table table = util.createTable(tableName, new byte[][] {A, B, C}); try { Append app = new Append(Bytes.toBytes(0)); app.add(A, A, A); @@ -341,7 +342,7 @@ public class TestRegionObserverInterface { new Boolean[] {false, false, false, false} ); - HTable table = new HTable(util.getConfiguration(), tableName); + Table table = new HTable(util.getConfiguration(), tableName); Put put = new Put(ROW); put.add(A, A, A); table.put(put); @@ -391,7 +392,7 @@ public class TestRegionObserverInterface { new Boolean[] {false, false} ); - HTable table = new HTable(util.getConfiguration(), tableName); + Table table = new HTable(util.getConfiguration(), tableName); Put put = new Put(ROW); put.add(A, A, A); table.put(put); @@ -498,7 +499,7 @@ public class TestRegionObserverInterface { htd.addCoprocessor(EvenOnlyCompactor.class.getName()); admin.createTable(htd); - HTable table = new HTable(util.getConfiguration(), compactTable); + Table table = new HTable(util.getConfiguration(), compactTable); for (long i=1; i<=10; i++) { byte[] iBytes = Bytes.toBytes(i); Put put = new Put(iBytes); @@ -560,7 +561,7 @@ public class TestRegionObserverInterface { String testName = TestRegionObserverInterface.class.getName()+".bulkLoadHFileTest"; TableName tableName = TableName.valueOf(TEST_TABLE.getNameAsString() + ".bulkLoadHFileTest"); Configuration conf = util.getConfiguration(); - HTable table = util.createTable(tableName, new byte[][] {A, B, C}); + Table table = util.createTable(tableName, new byte[][] {A, B, C}); try { verifyMethodResult(SimpleRegionObserver.class, new String[] {"hadPreBulkLoadHFile", "hadPostBulkLoadHFile"}, diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/coprocessor/TestRegionObserverScannerOpenHook.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/coprocessor/TestRegionObserverScannerOpenHook.java index 00f7c49b14f..3365a95a1c8 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/coprocessor/TestRegionObserverScannerOpenHook.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/coprocessor/TestRegionObserverScannerOpenHook.java @@ -48,6 +48,7 @@ import org.apache.hadoop.hbase.client.IsolationLevel; import org.apache.hadoop.hbase.client.Put; import org.apache.hadoop.hbase.client.Result; import org.apache.hadoop.hbase.client.Scan; +import org.apache.hadoop.hbase.client.Table; import org.apache.hadoop.hbase.filter.FilterBase; import org.apache.hadoop.hbase.regionserver.HRegion; import org.apache.hadoop.hbase.regionserver.HRegionServer; @@ -257,7 +258,7 @@ public class TestRegionObserverScannerOpenHook { Admin admin = UTIL.getHBaseAdmin(); admin.createTable(desc); - HTable table = new HTable(conf, desc.getTableName()); + Table table = new HTable(conf, desc.getTableName()); // put a row and flush it to disk Put put = new Put(ROW); diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/coprocessor/TestRegionServerObserver.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/coprocessor/TestRegionServerObserver.java index 5920bb5b2e9..f9d15c8c48b 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/coprocessor/TestRegionServerObserver.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/coprocessor/TestRegionServerObserver.java @@ -36,6 +36,7 @@ import org.apache.hadoop.hbase.MediumTests; import org.apache.hadoop.hbase.MiniHBaseCluster; import org.apache.hadoop.hbase.TableName; import org.apache.hadoop.hbase.MetaTableAccessor; +import org.apache.hadoop.hbase.client.Admin; import org.apache.hadoop.hbase.client.HBaseAdmin; import org.apache.hadoop.hbase.client.Mutation; import org.apache.hadoop.hbase.regionserver.HRegion; @@ -74,7 +75,7 @@ public class TestRegionServerObserver { // Start the cluster HBaseTestingUtility TEST_UTIL = new HBaseTestingUtility(conf); TEST_UTIL.startMiniCluster(NUM_MASTERS, NUM_RS); - HBaseAdmin admin = new HBaseAdmin(conf); + Admin admin = new HBaseAdmin(conf); try { MiniHBaseCluster cluster = TEST_UTIL.getHBaseCluster(); HRegionServer regionServer = cluster.getRegionServer(0); diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/coprocessor/TestRowProcessorEndpoint.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/coprocessor/TestRowProcessorEndpoint.java index 085348e0147..76612ce1823 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/coprocessor/TestRowProcessorEndpoint.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/coprocessor/TestRowProcessorEndpoint.java @@ -34,6 +34,7 @@ import java.util.concurrent.atomic.AtomicInteger; import java.util.concurrent.atomic.AtomicLong; import org.apache.hadoop.hbase.TableName; +import org.apache.hadoop.hbase.client.Table; import org.apache.hadoop.hbase.util.ByteStringer; import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.hbase.Cell; @@ -44,7 +45,6 @@ import org.apache.hadoop.hbase.KeyValue; import org.apache.hadoop.hbase.MediumTests; import org.apache.hadoop.hbase.client.Delete; import org.apache.hadoop.hbase.client.Get; -import org.apache.hadoop.hbase.client.HTable; import org.apache.hadoop.hbase.client.IsolationLevel; import org.apache.hadoop.hbase.client.Mutation; import org.apache.hadoop.hbase.client.Put; @@ -107,7 +107,7 @@ public class TestRowProcessorEndpoint { private static volatile int expectedCounter = 0; private static int rowSize, row2Size; - private volatile static HTable table = null; + private volatile static Table table = null; private volatile static boolean swapped = false; private volatile CountDownLatch startSignal; private volatile CountDownLatch doneSignal; @@ -196,7 +196,7 @@ public class TestRowProcessorEndpoint { } } - private int incrementCounter(HTable table) throws Throwable { + private int incrementCounter(Table table) throws Throwable { CoprocessorRpcChannel channel = table.coprocessorService(ROW); RowProcessorEndpoint.IncrementCounterProcessor processor = new RowProcessorEndpoint.IncrementCounterProcessor(ROW); @@ -259,7 +259,7 @@ public class TestRowProcessorEndpoint { } } - private void swapRows(HTable table) throws Throwable { + private void swapRows(Table table) throws Throwable { CoprocessorRpcChannel channel = table.coprocessorService(ROW); RowProcessorEndpoint.RowSwapProcessor processor = new RowProcessorEndpoint.RowSwapProcessor(ROW, ROW2); diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/filter/TestColumnRangeFilter.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/filter/TestColumnRangeFilter.java index 20135ddede7..c501149140a 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/filter/TestColumnRangeFilter.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/filter/TestColumnRangeFilter.java @@ -19,7 +19,6 @@ package org.apache.hadoop.hbase.filter; import static org.junit.Assert.*; -import java.io.IOException; import java.util.ArrayList; import java.util.HashMap; import java.util.HashSet; @@ -30,14 +29,12 @@ import java.util.Set; import org.apache.commons.logging.Log; import org.apache.commons.logging.LogFactory; import org.apache.hadoop.hbase.*; -import org.apache.hadoop.hbase.client.HTable; import org.apache.hadoop.hbase.client.Put; import org.apache.hadoop.hbase.client.Result; import org.apache.hadoop.hbase.client.ResultScanner; import org.apache.hadoop.hbase.client.Scan; import org.apache.hadoop.hbase.client.Durability; -import org.apache.hadoop.hbase.regionserver.HRegion; -import org.apache.hadoop.hbase.regionserver.InternalScanner; +import org.apache.hadoop.hbase.client.Table; import org.apache.hadoop.hbase.util.Bytes; import org.junit.Test; import org.junit.After; @@ -161,7 +158,7 @@ public class TestColumnRangeFilter { public void TestColumnRangeFilterClient() throws Exception { String family = "Family"; String table = "TestColumnRangeFilterClient"; - HTable ht = TEST_UTIL.createTable(Bytes.toBytes(table), + Table ht = TEST_UTIL.createTable(Bytes.toBytes(table), Bytes.toBytes(family), Integer.MAX_VALUE); List rows = generateRandomWords(10, 8); diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/filter/TestFilterWithScanLimits.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/filter/TestFilterWithScanLimits.java index a35d5c570e7..3955d36675f 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/filter/TestFilterWithScanLimits.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/filter/TestFilterWithScanLimits.java @@ -49,6 +49,7 @@ import org.apache.hadoop.hbase.client.Result; import org.apache.hadoop.hbase.client.ResultScanner; import org.apache.hadoop.hbase.client.Scan; import org.apache.hadoop.hbase.client.ScannerCallable; +import org.apache.hadoop.hbase.client.Table; import org.apache.hadoop.hbase.ipc.RpcClient; import org.apache.hadoop.hbase.ipc.RpcServer; import org.apache.hadoop.hbase.util.Bytes; @@ -85,7 +86,7 @@ public class TestFilterWithScanLimits { // add filter after batch defined scan.setFilter(filter); - HTable table = new HTable(conf, name); + Table table = new HTable(conf, name); ResultScanner scanner = table.getScanner(scan); // Expect to get following row // row2 => , , @@ -111,7 +112,7 @@ public class TestFilterWithScanLimits { private static void prepareData() { try { - HTable table = new HTable(TestFilterWithScanLimits.conf, name); + Table table = new HTable(TestFilterWithScanLimits.conf, name); assertTrue("Fail to create the table", admin.tableExists(name)); List puts = new ArrayList(); diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/filter/TestFilterWrapper.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/filter/TestFilterWrapper.java index 9587aa3a952..46a70429e17 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/filter/TestFilterWrapper.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/filter/TestFilterWrapper.java @@ -33,7 +33,6 @@ import org.apache.hadoop.hbase.HBaseTestingUtility; import org.apache.hadoop.hbase.HColumnDescriptor; import org.apache.hadoop.hbase.HConstants; import org.apache.hadoop.hbase.HTableDescriptor; -import org.apache.hadoop.hbase.KeyValue; import org.apache.hadoop.hbase.TableName; import org.apache.hadoop.hbase.MasterNotRunningException; import org.apache.hadoop.hbase.ZooKeeperConnectionException; @@ -43,6 +42,7 @@ import org.apache.hadoop.hbase.client.Put; import org.apache.hadoop.hbase.client.Result; import org.apache.hadoop.hbase.client.ResultScanner; import org.apache.hadoop.hbase.client.Scan; +import org.apache.hadoop.hbase.client.Table; import org.apache.hadoop.hbase.util.Bytes; import org.junit.AfterClass; @@ -83,7 +83,7 @@ public class TestFilterWrapper { FilterList filter = new FilterList(fs); scan.setFilter(filter); - HTable table = new HTable(conf, name); + Table table = new HTable(conf, name); ResultScanner scanner = table.getScanner(scan); // row2 (c1-c4) and row3(c1-c4) are returned @@ -110,7 +110,7 @@ public class TestFilterWrapper { private static void prepareData() { try { - HTable table = new HTable(TestFilterWrapper.conf, name); + Table table = new HTable(TestFilterWrapper.conf, name); assertTrue("Fail to create the table", admin.tableExists(name)); List puts = new ArrayList(); diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/filter/TestFuzzyRowAndColumnRangeFilter.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/filter/TestFuzzyRowAndColumnRangeFilter.java index 1f853ac7184..d64a5ad432d 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/filter/TestFuzzyRowAndColumnRangeFilter.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/filter/TestFuzzyRowAndColumnRangeFilter.java @@ -29,11 +29,11 @@ import org.apache.hadoop.hbase.CellUtil; import org.apache.hadoop.hbase.HBaseTestingUtility; import org.apache.hadoop.hbase.MediumTests; import org.apache.hadoop.hbase.client.Durability; -import org.apache.hadoop.hbase.client.HTable; import org.apache.hadoop.hbase.client.Put; import org.apache.hadoop.hbase.client.Result; import org.apache.hadoop.hbase.client.ResultScanner; 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.Pair; import org.junit.After; @@ -88,7 +88,7 @@ public class TestFuzzyRowAndColumnRangeFilter { public void Test() throws Exception { String cf = "f"; String table = "TestFuzzyAndColumnRangeFilterClient"; - HTable ht = TEST_UTIL.createTable(Bytes.toBytes(table), + Table ht = TEST_UTIL.createTable(Bytes.toBytes(table), Bytes.toBytes(cf), Integer.MAX_VALUE); // 10 byte row key - (2 bytes 4 bytes 4 bytes) @@ -128,7 +128,7 @@ public class TestFuzzyRowAndColumnRangeFilter { runTest(ht, 1, 8); } - private void runTest(HTable hTable, int cqStart, int expectedSize) throws IOException { + private void runTest(Table hTable, int cqStart, int expectedSize) throws IOException { // [0, 2, ?, ?, ?, ?, 0, 0, 0, 1] byte[] fuzzyKey = new byte[10]; ByteBuffer buf = ByteBuffer.wrap(fuzzyKey); @@ -150,7 +150,7 @@ public class TestFuzzyRowAndColumnRangeFilter { runScanner(hTable, expectedSize, columnRangeFilter, fuzzyRowFilter); } - private void runScanner(HTable hTable, int expectedSize, Filter... filters) throws IOException { + private void runScanner(Table hTable, int expectedSize, Filter... filters) throws IOException { String cf = "f"; Scan scan = new Scan(); scan.addFamily(cf.getBytes()); diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/fs/TestBlockReorder.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/fs/TestBlockReorder.java index 49e346584f7..f83fbf36265 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/fs/TestBlockReorder.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/fs/TestBlockReorder.java @@ -39,8 +39,8 @@ import org.apache.hadoop.hbase.HBaseTestingUtility; import org.apache.hadoop.hbase.HConstants; import org.apache.hadoop.hbase.LargeTests; import org.apache.hadoop.hbase.MiniHBaseCluster; -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.regionserver.HRegionServer; import org.apache.hadoop.hbase.regionserver.wal.HLogUtil; import org.apache.hadoop.hbase.util.FSUtils; @@ -259,7 +259,7 @@ public class TestBlockReorder { // We use the regionserver file system & conf as we expect it to have the hook. conf = targetRs.getConfiguration(); HFileSystem rfs = (HFileSystem) targetRs.getFileSystem(); - HTable h = htu.createTable("table".getBytes(), sb); + Table h = htu.createTable("table".getBytes(), sb); // Now, we have 4 datanodes and a replication count of 3. So we don't know if the datanode // with the same node will be used. We can't really stop an existing datanode, this would diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/io/encoding/TestChangingEncoding.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/io/encoding/TestChangingEncoding.java index 0ceb9532a80..d4195106695 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/io/encoding/TestChangingEncoding.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/io/encoding/TestChangingEncoding.java @@ -42,6 +42,7 @@ import org.apache.hadoop.hbase.client.HBaseAdmin; import org.apache.hadoop.hbase.client.HTable; import org.apache.hadoop.hbase.client.Put; import org.apache.hadoop.hbase.client.Result; +import org.apache.hadoop.hbase.client.Table; import org.apache.hadoop.hbase.regionserver.HRegionServer; import org.apache.hadoop.hbase.util.Bytes; import org.apache.hadoop.hbase.util.Threads; @@ -137,7 +138,7 @@ public class TestChangingEncoding { static void writeTestDataBatch(Configuration conf, String tableName, int batchId) throws Exception { LOG.debug("Writing test data batch " + batchId); - HTable table = new HTable(conf, tableName); + Table table = new HTable(conf, tableName); table.setAutoFlushTo(false); for (int i = 0; i < NUM_ROWS_PER_BATCH; ++i) { Put put = new Put(getRowKey(batchId, i)); @@ -155,7 +156,7 @@ public class TestChangingEncoding { static void verifyTestDataBatch(Configuration conf, String tableName, int batchId) throws Exception { LOG.debug("Verifying test data batch " + batchId); - HTable table = new HTable(conf, tableName); + Table table = new HTable(conf, tableName); for (int i = 0; i < NUM_ROWS_PER_BATCH; ++i) { Get get = new Get(getRowKey(batchId, i)); Result result = table.get(get); diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/io/encoding/TestPrefixTree.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/io/encoding/TestPrefixTree.java index d19437e7e90..710aed9f22f 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/io/encoding/TestPrefixTree.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/io/encoding/TestPrefixTree.java @@ -34,6 +34,7 @@ import org.apache.hadoop.hbase.client.Put; import org.apache.hadoop.hbase.client.Result; import org.apache.hadoop.hbase.client.ResultScanner; import org.apache.hadoop.hbase.client.Scan; +import org.apache.hadoop.hbase.client.Table; import org.apache.hadoop.hbase.util.Bytes; import org.junit.AfterClass; import org.junit.BeforeClass; @@ -79,7 +80,7 @@ public class TestPrefixTree { @Test public void testHBASE11728() throws Exception { TableName tableName = TableName.valueOf(TEST_NAME.getMethodName()); - HTable table = null; + Table table = null; try { Admin hBaseAdmin = TEST_UTIL.getHBaseAdmin(); HColumnDescriptor colDesc = new HColumnDescriptor(fam); diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/mapred/TestTableInputFormat.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/mapred/TestTableInputFormat.java index 88ffd9ffbf4..1cdde7c8728 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/mapred/TestTableInputFormat.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/mapred/TestTableInputFormat.java @@ -35,11 +35,11 @@ import java.util.Map; import org.apache.commons.logging.Log; import org.apache.commons.logging.LogFactory; import org.apache.hadoop.hbase.*; -import org.apache.hadoop.hbase.client.HTable; import org.apache.hadoop.hbase.client.Put; import org.apache.hadoop.hbase.client.Result; import org.apache.hadoop.hbase.client.ResultScanner; import org.apache.hadoop.hbase.client.Scan; +import org.apache.hadoop.hbase.client.Table; import org.apache.hadoop.hbase.io.ImmutableBytesWritable; import org.apache.hadoop.hbase.util.Bytes; import org.junit.AfterClass; @@ -88,8 +88,8 @@ public class TestTableInputFormat { * @return * @throws IOException */ - public static HTable createTable(byte[] tableName) throws IOException { - HTable table = UTIL.createTable(tableName, FAMILY); + public static Table createTable(byte[] tableName) throws IOException { + Table table = UTIL.createTable(tableName, FAMILY); Put p = new Put("aaa".getBytes()); p.add(FAMILY, null, "value aaa".getBytes()); table.put(p); @@ -124,7 +124,7 @@ public class TestTableInputFormat { * @param table * @throws IOException */ - static void runTestMapred(HTable table) throws IOException { + static void runTestMapred(Table table) throws IOException { org.apache.hadoop.hbase.mapred.TableRecordReader trr = new org.apache.hadoop.hbase.mapred.TableRecordReader(); trr.setStartRow("aaa".getBytes()); @@ -157,7 +157,7 @@ public class TestTableInputFormat { * @throws IOException * @throws InterruptedException */ - static void runTestMapreduce(HTable table) throws IOException, + static void runTestMapreduce(Table table) throws IOException, InterruptedException { org.apache.hadoop.hbase.mapreduce.TableRecordReaderImpl trr = new org.apache.hadoop.hbase.mapreduce.TableRecordReaderImpl(); @@ -194,7 +194,7 @@ public class TestTableInputFormat { * * @throws IOException */ - static HTable createIOEScannerTable(byte[] name, final int failCnt) + static Table createIOEScannerTable(byte[] name, final int failCnt) throws IOException { // build up a mock scanner stuff to fail the first time Answer a = new Answer() { @@ -218,7 +218,7 @@ public class TestTableInputFormat { } }; - HTable htable = spy(createTable(name)); + Table htable = spy(createTable(name)); doAnswer(a).when(htable).getScanner((Scan) anyObject()); return htable; } @@ -229,7 +229,7 @@ public class TestTableInputFormat { * * @throws IOException */ - static HTable createDNRIOEScannerTable(byte[] name, final int failCnt) + static Table createDNRIOEScannerTable(byte[] name, final int failCnt) throws IOException { // build up a mock scanner stuff to fail the first time Answer a = new Answer() { @@ -256,7 +256,7 @@ public class TestTableInputFormat { } }; - HTable htable = spy(createTable(name)); + Table htable = spy(createTable(name)); doAnswer(a).when(htable).getScanner((Scan) anyObject()); return htable; } @@ -268,7 +268,7 @@ public class TestTableInputFormat { */ @Test public void testTableRecordReader() throws IOException { - HTable table = createTable("table1".getBytes()); + Table table = createTable("table1".getBytes()); runTestMapred(table); } @@ -279,7 +279,7 @@ public class TestTableInputFormat { */ @Test public void testTableRecordReaderScannerFail() throws IOException { - HTable htable = createIOEScannerTable("table2".getBytes(), 1); + Table htable = createIOEScannerTable("table2".getBytes(), 1); runTestMapred(htable); } @@ -290,7 +290,7 @@ public class TestTableInputFormat { */ @Test(expected = IOException.class) public void testTableRecordReaderScannerFailTwice() throws IOException { - HTable htable = createIOEScannerTable("table3".getBytes(), 2); + Table htable = createIOEScannerTable("table3".getBytes(), 2); runTestMapred(htable); } @@ -302,7 +302,7 @@ public class TestTableInputFormat { */ @Test public void testTableRecordReaderScannerTimeout() throws IOException { - HTable htable = createDNRIOEScannerTable("table4".getBytes(), 1); + Table htable = createDNRIOEScannerTable("table4".getBytes(), 1); runTestMapred(htable); } @@ -314,7 +314,7 @@ public class TestTableInputFormat { */ @Test(expected = org.apache.hadoop.hbase.DoNotRetryIOException.class) public void testTableRecordReaderScannerTimeoutTwice() throws IOException { - HTable htable = createDNRIOEScannerTable("table5".getBytes(), 2); + Table htable = createDNRIOEScannerTable("table5".getBytes(), 2); runTestMapred(htable); } @@ -327,7 +327,7 @@ public class TestTableInputFormat { @Test public void testTableRecordReaderMapreduce() throws IOException, InterruptedException { - HTable table = createTable("table1-mr".getBytes()); + Table table = createTable("table1-mr".getBytes()); runTestMapreduce(table); } @@ -340,7 +340,7 @@ public class TestTableInputFormat { @Test public void testTableRecordReaderScannerFailMapreduce() throws IOException, InterruptedException { - HTable htable = createIOEScannerTable("table2-mr".getBytes(), 1); + Table htable = createIOEScannerTable("table2-mr".getBytes(), 1); runTestMapreduce(htable); } @@ -353,7 +353,7 @@ public class TestTableInputFormat { @Test(expected = IOException.class) public void testTableRecordReaderScannerFailMapreduceTwice() throws IOException, InterruptedException { - HTable htable = createIOEScannerTable("table3-mr".getBytes(), 2); + Table htable = createIOEScannerTable("table3-mr".getBytes(), 2); runTestMapreduce(htable); } @@ -367,7 +367,7 @@ public class TestTableInputFormat { @Test public void testTableRecordReaderScannerTimeoutMapreduce() throws IOException, InterruptedException { - HTable htable = createDNRIOEScannerTable("table4-mr".getBytes(), 1); + Table htable = createDNRIOEScannerTable("table4-mr".getBytes(), 1); runTestMapreduce(htable); } @@ -381,7 +381,7 @@ public class TestTableInputFormat { @Test(expected = org.apache.hadoop.hbase.DoNotRetryIOException.class) public void testTableRecordReaderScannerTimeoutMapreduceTwice() throws IOException, InterruptedException { - HTable htable = createDNRIOEScannerTable("table5-mr".getBytes(), 2); + Table htable = createDNRIOEScannerTable("table5-mr".getBytes(), 2); runTestMapreduce(htable); } diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/mapred/TestTableMapReduceUtil.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/mapred/TestTableMapReduceUtil.java index 99fc48d38f0..e527f9772a4 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/mapred/TestTableMapReduceUtil.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/mapred/TestTableMapReduceUtil.java @@ -34,9 +34,9 @@ import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.fs.FileUtil; import org.apache.hadoop.hbase.HBaseTestingUtility; import org.apache.hadoop.hbase.LargeTests; -import org.apache.hadoop.hbase.client.HTable; import org.apache.hadoop.hbase.client.Put; import org.apache.hadoop.hbase.client.Result; +import org.apache.hadoop.hbase.client.Table; import org.apache.hadoop.hbase.io.ImmutableBytesWritable; import org.apache.hadoop.hbase.util.Bytes; import org.apache.hadoop.mapred.JobClient; @@ -61,7 +61,7 @@ public class TestTableMapReduceUtil { private static final Log LOG = LogFactory .getLog(TestTableMapReduceUtil.class); - private static HTable presidentsTable; + private static Table presidentsTable; private static final String TABLE_NAME = "People"; private static final byte[] COLUMN_FAMILY = Bytes.toBytes("info"); @@ -104,13 +104,13 @@ public class TestTableMapReduceUtil { LOG.info("before done"); } - public static HTable createAndFillTable(byte[] tableName) throws IOException { - HTable table = UTIL.createTable(tableName, COLUMN_FAMILY); + public static Table createAndFillTable(byte[] tableName) throws IOException { + Table table = UTIL.createTable(tableName, COLUMN_FAMILY); createPutCommand(table); return table; } - private static void createPutCommand(HTable table) throws IOException { + private static void createPutCommand(Table table) throws IOException { for (String president : presidentsRowKeys) { if (presidentNames.hasNext()) { Put p = new Put(Bytes.toBytes(president)); diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/mapreduce/TestCellCounter.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/mapreduce/TestCellCounter.java index 1ccc57d7928..72f48b1a246 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/mapreduce/TestCellCounter.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/mapreduce/TestCellCounter.java @@ -25,8 +25,8 @@ import org.apache.hadoop.fs.LocalFileSystem; import org.apache.hadoop.fs.Path; import org.apache.hadoop.hbase.HBaseTestingUtility; import org.apache.hadoop.hbase.LargeTests; -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.util.Bytes; import org.apache.hadoop.hbase.util.LauncherSecurityManager; import org.apache.hadoop.mapreduce.Job; @@ -80,7 +80,7 @@ public class TestCellCounter { public void testCellCounter() throws Exception { String sourceTable = "sourceTable"; byte[][] families = { FAMILY_A, FAMILY_B }; - HTable t = UTIL.createTable(Bytes.toBytes(sourceTable), families); + Table t = UTIL.createTable(Bytes.toBytes(sourceTable), families); try{ Put p = new Put(ROW1); p.add(FAMILY_A, QUALIFIER, now, Bytes.toBytes("Data11")); diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/mapreduce/TestCopyTable.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/mapreduce/TestCopyTable.java index 6163bb9afa7..8d171a614a1 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/mapreduce/TestCopyTable.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/mapreduce/TestCopyTable.java @@ -33,9 +33,9 @@ import org.apache.hadoop.hbase.HBaseTestingUtility; import org.apache.hadoop.hbase.LargeTests; import org.apache.hadoop.hbase.MiniHBaseCluster; import org.apache.hadoop.hbase.client.Get; -import org.apache.hadoop.hbase.client.HTable; import org.apache.hadoop.hbase.client.Put; import org.apache.hadoop.hbase.client.Result; +import org.apache.hadoop.hbase.client.Table; import org.apache.hadoop.hbase.util.Bytes; import org.apache.hadoop.hbase.util.LauncherSecurityManager; import org.apache.hadoop.mapreduce.Job; @@ -84,8 +84,8 @@ public class TestCopyTable { final byte[] FAMILY = Bytes.toBytes("family"); final byte[] COLUMN1 = Bytes.toBytes("c1"); - HTable t1 = TEST_UTIL.createTable(TABLENAME1, FAMILY); - HTable t2 = TEST_UTIL.createTable(TABLENAME2, FAMILY); + Table t1 = TEST_UTIL.createTable(TABLENAME1, FAMILY); + Table t2 = TEST_UTIL.createTable(TABLENAME2, FAMILY); // put rows into the first table for (int i = 0; i < 10; i++) { @@ -125,8 +125,8 @@ public class TestCopyTable { final byte[] ROW1 = Bytes.toBytes("row1"); final byte[] ROW2 = Bytes.toBytes("row2"); - HTable t1 = TEST_UTIL.createTable(TABLENAME1, FAMILY); - HTable t2 = TEST_UTIL.createTable(TABLENAME2, FAMILY); + Table t1 = TEST_UTIL.createTable(TABLENAME1, FAMILY); + Table t2 = TEST_UTIL.createTable(TABLENAME2, FAMILY); // put rows into the first table Put p = new Put(ROW0); @@ -176,8 +176,8 @@ public class TestCopyTable { byte[][] families = { FAMILY_A, FAMILY_B }; - HTable t = TEST_UTIL.createTable(Bytes.toBytes(sourceTable), families); - HTable t2 = TEST_UTIL.createTable(Bytes.toBytes(targetTable), families); + Table t = TEST_UTIL.createTable(Bytes.toBytes(sourceTable), families); + Table t2 = TEST_UTIL.createTable(Bytes.toBytes(targetTable), families); Put p = new Put(ROW1); p.add(FAMILY_A, QUALIFIER, Bytes.toBytes("Data11")); p.add(FAMILY_B, QUALIFIER, Bytes.toBytes("Data12")); diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/mapreduce/TestHFileOutputFormat.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/mapreduce/TestHFileOutputFormat.java index a46660e183c..c257cf3f56b 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/mapreduce/TestHFileOutputFormat.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/mapreduce/TestHFileOutputFormat.java @@ -58,9 +58,11 @@ import org.apache.hadoop.hbase.TableName; import org.apache.hadoop.hbase.client.HBaseAdmin; import org.apache.hadoop.hbase.client.HTable; import org.apache.hadoop.hbase.client.Put; +import org.apache.hadoop.hbase.client.RegionLocator; import org.apache.hadoop.hbase.client.Result; import org.apache.hadoop.hbase.client.ResultScanner; import org.apache.hadoop.hbase.client.Scan; +import org.apache.hadoop.hbase.client.Table; import org.apache.hadoop.hbase.io.ImmutableBytesWritable; import org.apache.hadoop.hbase.io.compress.Compression; import org.apache.hadoop.hbase.io.compress.Compression.Algorithm; @@ -476,7 +478,7 @@ public class TestHFileOutputFormat { } /** - * Test for {@link HFileOutputFormat#configureCompression(HTable, + * Test for {@link HFileOutputFormat#configureCompression(org.apache.hadoop.hbase.client.Table, * Configuration)} and {@link HFileOutputFormat#createFamilyCompressionMap * (Configuration)}. * Tests that the compression map is correctly serialized into @@ -490,7 +492,7 @@ public class TestHFileOutputFormat { Configuration conf = new Configuration(this.util.getConfiguration()); Map familyToCompression = getMockColumnFamiliesForCompression(numCfs); - HTable table = Mockito.mock(HTable.class); + Table table = Mockito.mock(HTable.class); setupMockColumnFamiliesForCompression(table, familyToCompression); HFileOutputFormat.configureCompression(table, conf); @@ -508,7 +510,7 @@ public class TestHFileOutputFormat { } } - private void setupMockColumnFamiliesForCompression(HTable table, + private void setupMockColumnFamiliesForCompression(Table table, Map familyToCompression) throws IOException { HTableDescriptor mockTableDescriptor = new HTableDescriptor(TABLE_NAME); for (Entry entry : familyToCompression.entrySet()) { @@ -546,7 +548,7 @@ public class TestHFileOutputFormat { /** - * Test for {@link HFileOutputFormat#configureBloomType(HTable, + * Test for {@link HFileOutputFormat#configureBloomType(org.apache.hadoop.hbase.client.Table, * Configuration)} and {@link HFileOutputFormat#createFamilyBloomTypeMap * (Configuration)}. * Tests that the compression map is correctly serialized into @@ -560,7 +562,7 @@ public class TestHFileOutputFormat { Configuration conf = new Configuration(this.util.getConfiguration()); Map familyToBloomType = getMockColumnFamiliesForBloomType(numCfs); - HTable table = Mockito.mock(HTable.class); + Table table = Mockito.mock(HTable.class); setupMockColumnFamiliesForBloomType(table, familyToBloomType); HFileOutputFormat.configureBloomType(table, conf); @@ -581,7 +583,7 @@ public class TestHFileOutputFormat { } } - private void setupMockColumnFamiliesForBloomType(HTable table, + private void setupMockColumnFamiliesForBloomType(Table table, Map familyToDataBlockEncoding) throws IOException { HTableDescriptor mockTableDescriptor = new HTableDescriptor(TABLE_NAME); for (Entry entry : familyToDataBlockEncoding.entrySet()) { @@ -617,7 +619,7 @@ public class TestHFileOutputFormat { } /** - * Test for {@link HFileOutputFormat#configureBlockSize(HTable, + * Test for {@link HFileOutputFormat#configureBlockSize(org.apache.hadoop.hbase.client.Table, * Configuration)} and {@link HFileOutputFormat#createFamilyBlockSizeMap * (Configuration)}. * Tests that the compression map is correctly serialized into @@ -631,7 +633,7 @@ public class TestHFileOutputFormat { Configuration conf = new Configuration(this.util.getConfiguration()); Map familyToBlockSize = getMockColumnFamiliesForBlockSize(numCfs); - HTable table = Mockito.mock(HTable.class); + Table table = Mockito.mock(HTable.class); setupMockColumnFamiliesForBlockSize(table, familyToBlockSize); HFileOutputFormat.configureBlockSize(table, conf); @@ -653,7 +655,7 @@ public class TestHFileOutputFormat { } } - private void setupMockColumnFamiliesForBlockSize(HTable table, + private void setupMockColumnFamiliesForBlockSize(Table table, Map familyToDataBlockEncoding) throws IOException { HTableDescriptor mockTableDescriptor = new HTableDescriptor(TABLE_NAME); for (Entry entry : familyToDataBlockEncoding.entrySet()) { @@ -693,7 +695,7 @@ public class TestHFileOutputFormat { } /** - * Test for {@link HFileOutputFormat#configureDataBlockEncoding(HTable, + * Test for {@link HFileOutputFormat#configureDataBlockEncoding(org.apache.hadoop.hbase.client.Table, * Configuration)} and {@link HFileOutputFormat#createFamilyDataBlockEncodingMap * (Configuration)}. * Tests that the compression map is correctly serialized into @@ -707,7 +709,7 @@ public class TestHFileOutputFormat { Configuration conf = new Configuration(this.util.getConfiguration()); Map familyToDataBlockEncoding = getMockColumnFamiliesForDataBlockEncoding(numCfs); - HTable table = Mockito.mock(HTable.class); + Table table = Mockito.mock(HTable.class); setupMockColumnFamiliesForDataBlockEncoding(table, familyToDataBlockEncoding); HFileOutputFormat.configureDataBlockEncoding(table, conf); @@ -728,7 +730,7 @@ public class TestHFileOutputFormat { } } - private void setupMockColumnFamiliesForDataBlockEncoding(HTable table, + private void setupMockColumnFamiliesForDataBlockEncoding(Table table, Map familyToDataBlockEncoding) throws IOException { HTableDescriptor mockTableDescriptor = new HTableDescriptor(TABLE_NAME); for (Entry entry : familyToDataBlockEncoding.entrySet()) { @@ -767,7 +769,7 @@ public class TestHFileOutputFormat { return familyToDataBlockEncoding; } - private void setupMockStartKeys(HTable table) throws IOException { + private void setupMockStartKeys(RegionLocator table) throws IOException { byte[][] mockKeys = new byte[][] { HConstants.EMPTY_BYTE_ARRAY, Bytes.toBytes("aaa"), diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/mapreduce/TestHFileOutputFormat2.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/mapreduce/TestHFileOutputFormat2.java index 76db299184b..edfd3395c08 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/mapreduce/TestHFileOutputFormat2.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/mapreduce/TestHFileOutputFormat2.java @@ -56,9 +56,11 @@ import org.apache.hadoop.hbase.TableName; import org.apache.hadoop.hbase.client.HBaseAdmin; import org.apache.hadoop.hbase.client.HTable; import org.apache.hadoop.hbase.client.Put; +import org.apache.hadoop.hbase.client.RegionLocator; import org.apache.hadoop.hbase.client.Result; import org.apache.hadoop.hbase.client.ResultScanner; import org.apache.hadoop.hbase.client.Scan; +import org.apache.hadoop.hbase.client.Table; import org.apache.hadoop.hbase.io.ImmutableBytesWritable; import org.apache.hadoop.hbase.io.compress.Compression; import org.apache.hadoop.hbase.io.compress.Compression.Algorithm; @@ -475,7 +477,7 @@ public class TestHFileOutputFormat2 { } /** - * Test for {@link HFileOutputFormat2#configureCompression(HTable, + * Test for {@link HFileOutputFormat2#configureCompression(org.apache.hadoop.hbase.client.Table, * Configuration)} and {@link HFileOutputFormat2#createFamilyCompressionMap * (Configuration)}. * Tests that the compression map is correctly serialized into @@ -489,7 +491,7 @@ public class TestHFileOutputFormat2 { Configuration conf = new Configuration(this.util.getConfiguration()); Map familyToCompression = getMockColumnFamiliesForCompression(numCfs); - HTable table = Mockito.mock(HTable.class); + Table table = Mockito.mock(HTable.class); setupMockColumnFamiliesForCompression(table, familyToCompression); HFileOutputFormat2.configureCompression(table, conf); @@ -507,7 +509,7 @@ public class TestHFileOutputFormat2 { } } - private void setupMockColumnFamiliesForCompression(HTable table, + private void setupMockColumnFamiliesForCompression(Table table, Map familyToCompression) throws IOException { HTableDescriptor mockTableDescriptor = new HTableDescriptor(TABLE_NAME); for (Entry entry : familyToCompression.entrySet()) { @@ -546,7 +548,7 @@ public class TestHFileOutputFormat2 { /** - * Test for {@link HFileOutputFormat2#configureBloomType(HTable, + * Test for {@link HFileOutputFormat2#configureBloomType(org.apache.hadoop.hbase.client.Table, * Configuration)} and {@link HFileOutputFormat2#createFamilyBloomTypeMap * (Configuration)}. * Tests that the compression map is correctly serialized into @@ -560,7 +562,7 @@ public class TestHFileOutputFormat2 { Configuration conf = new Configuration(this.util.getConfiguration()); Map familyToBloomType = getMockColumnFamiliesForBloomType(numCfs); - HTable table = Mockito.mock(HTable.class); + Table table = Mockito.mock(HTable.class); setupMockColumnFamiliesForBloomType(table, familyToBloomType); HFileOutputFormat2.configureBloomType(table, conf); @@ -581,7 +583,7 @@ public class TestHFileOutputFormat2 { } } - private void setupMockColumnFamiliesForBloomType(HTable table, + private void setupMockColumnFamiliesForBloomType(Table table, Map familyToDataBlockEncoding) throws IOException { HTableDescriptor mockTableDescriptor = new HTableDescriptor(TABLE_NAME); for (Entry entry : familyToDataBlockEncoding.entrySet()) { @@ -617,7 +619,7 @@ public class TestHFileOutputFormat2 { } /** - * Test for {@link HFileOutputFormat2#configureBlockSize(HTable, + * Test for {@link HFileOutputFormat2#configureBlockSize(org.apache.hadoop.hbase.client.Table, * Configuration)} and {@link HFileOutputFormat2#createFamilyBlockSizeMap * (Configuration)}. * Tests that the compression map is correctly serialized into @@ -631,7 +633,7 @@ public class TestHFileOutputFormat2 { Configuration conf = new Configuration(this.util.getConfiguration()); Map familyToBlockSize = getMockColumnFamiliesForBlockSize(numCfs); - HTable table = Mockito.mock(HTable.class); + Table table = Mockito.mock(HTable.class); setupMockColumnFamiliesForBlockSize(table, familyToBlockSize); HFileOutputFormat2.configureBlockSize(table, conf); @@ -653,7 +655,7 @@ public class TestHFileOutputFormat2 { } } - private void setupMockColumnFamiliesForBlockSize(HTable table, + private void setupMockColumnFamiliesForBlockSize(Table table, Map familyToDataBlockEncoding) throws IOException { HTableDescriptor mockTableDescriptor = new HTableDescriptor(TABLE_NAME); for (Entry entry : familyToDataBlockEncoding.entrySet()) { @@ -693,7 +695,7 @@ public class TestHFileOutputFormat2 { } /** - * Test for {@link HFileOutputFormat2#configureDataBlockEncoding(HTable, + * Test for {@link HFileOutputFormat2#configureDataBlockEncoding(org.apache.hadoop.hbase.client.Table, * Configuration)} and {@link HFileOutputFormat2#createFamilyDataBlockEncodingMap * (Configuration)}. * Tests that the compression map is correctly serialized into @@ -707,7 +709,7 @@ public class TestHFileOutputFormat2 { Configuration conf = new Configuration(this.util.getConfiguration()); Map familyToDataBlockEncoding = getMockColumnFamiliesForDataBlockEncoding(numCfs); - HTable table = Mockito.mock(HTable.class); + Table table = Mockito.mock(HTable.class); setupMockColumnFamiliesForDataBlockEncoding(table, familyToDataBlockEncoding); HFileOutputFormat2.configureDataBlockEncoding(table, conf); @@ -728,7 +730,7 @@ public class TestHFileOutputFormat2 { } } - private void setupMockColumnFamiliesForDataBlockEncoding(HTable table, + private void setupMockColumnFamiliesForDataBlockEncoding(Table table, Map familyToDataBlockEncoding) throws IOException { HTableDescriptor mockTableDescriptor = new HTableDescriptor(TABLE_NAME); for (Entry entry : familyToDataBlockEncoding.entrySet()) { @@ -767,7 +769,7 @@ public class TestHFileOutputFormat2 { return familyToDataBlockEncoding; } - private void setupMockStartKeys(HTable table) throws IOException { + private void setupMockStartKeys(RegionLocator table) throws IOException { byte[][] mockKeys = new byte[][] { HConstants.EMPTY_BYTE_ARRAY, Bytes.toBytes("aaa"), diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/mapreduce/TestImportExport.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/mapreduce/TestImportExport.java index 7852fc525de..f89808f203e 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/mapreduce/TestImportExport.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/mapreduce/TestImportExport.java @@ -53,6 +53,7 @@ import org.apache.hadoop.hbase.client.Put; import org.apache.hadoop.hbase.client.Result; import org.apache.hadoop.hbase.client.ResultScanner; import org.apache.hadoop.hbase.client.Scan; +import org.apache.hadoop.hbase.client.Table; import org.apache.hadoop.hbase.filter.Filter; import org.apache.hadoop.hbase.filter.FilterBase; import org.apache.hadoop.hbase.filter.PrefixFilter; @@ -159,7 +160,7 @@ public class TestImportExport { @Test public void testSimpleCase() throws Exception { String EXPORT_TABLE = "exportSimpleCase"; - HTable t = UTIL.createTable(Bytes.toBytes(EXPORT_TABLE), FAMILYA, 3); + Table t = UTIL.createTable(Bytes.toBytes(EXPORT_TABLE), FAMILYA, 3); Put p = new Put(ROW1); p.add(FAMILYA, QUAL, now, QUAL); p.add(FAMILYA, QUAL, now+1, QUAL); @@ -222,7 +223,7 @@ public class TestImportExport { fs.copyFromLocalFile(importPath, new Path(FQ_OUTPUT_DIR + Path.SEPARATOR + "exportedTableIn94Format")); String IMPORT_TABLE = "importTableExportedFrom94"; - HTable t = UTIL.createTable(Bytes.toBytes(IMPORT_TABLE), Bytes.toBytes("f1"), 3); + Table t = UTIL.createTable(Bytes.toBytes(IMPORT_TABLE), Bytes.toBytes("f1"), 3); String[] args = new String[] { "-Dhbase.import.version=0.94" , IMPORT_TABLE, FQ_OUTPUT_DIR @@ -252,7 +253,7 @@ public class TestImportExport { .setMaxVersions(1) ); UTIL.getHBaseAdmin().createTable(desc); - HTable t = new HTable(UTIL.getConfiguration(), BATCH_TABLE); + Table t = new HTable(UTIL.getConfiguration(), BATCH_TABLE); Put p = new Put(ROW1); p.add(FAMILYA, QUAL, now, QUAL); @@ -283,7 +284,7 @@ public class TestImportExport { .setKeepDeletedCells(true) ); UTIL.getHBaseAdmin().createTable(desc); - HTable t = new HTable(UTIL.getConfiguration(), EXPORT_TABLE); + Table t = new HTable(UTIL.getConfiguration(), EXPORT_TABLE); Put p = new Put(ROW1); p.add(FAMILYA, QUAL, now, QUAL); @@ -349,7 +350,7 @@ public class TestImportExport { HTableDescriptor desc = new HTableDescriptor(TableName.valueOf(EXPORT_TABLE)); desc.addFamily(new HColumnDescriptor(FAMILYA).setMaxVersions(5)); UTIL.getHBaseAdmin().createTable(desc); - HTable exportTable = new HTable(UTIL.getConfiguration(), EXPORT_TABLE); + Table exportTable = new HTable(UTIL.getConfiguration(), EXPORT_TABLE); Put p = new Put(ROW1); p.add(FAMILYA, QUAL, now, QUAL); @@ -376,7 +377,7 @@ public class TestImportExport { desc.addFamily(new HColumnDescriptor(FAMILYA).setMaxVersions(5)); UTIL.getHBaseAdmin().createTable(desc); - HTable importTable = new HTable(UTIL.getConfiguration(), IMPORT_TABLE); + Table importTable = new HTable(UTIL.getConfiguration(), IMPORT_TABLE); args = new String[] { "-D" + Import.FILTER_CLASS_CONF_KEY + "=" + PrefixFilter.class.getName(), "-D" + Import.FILTER_ARGS_CONF_KEY + "=" + Bytes.toString(ROW1), IMPORT_TABLE, FQ_OUTPUT_DIR, "1000" }; @@ -410,7 +411,7 @@ public class TestImportExport { * @return * @throws IOException */ - private int getCount(HTable table, Filter filter) throws IOException { + private int getCount(Table table, Filter filter) throws IOException { Scan scan = new Scan(); scan.setFilter(filter); ResultScanner results = table.getScanner(scan); @@ -542,7 +543,7 @@ public class TestImportExport { public void testDurability() throws IOException, InterruptedException, ClassNotFoundException { // Create an export table. String exportTableName = "exporttestDurability"; - HTable exportTable = UTIL.createTable(Bytes.toBytes(exportTableName), FAMILYA, 3); + Table exportTable = UTIL.createTable(Bytes.toBytes(exportTableName), FAMILYA, 3); // Insert some data Put put = new Put(ROW1); @@ -563,7 +564,7 @@ public class TestImportExport { // Create the table for import String importTableName = "importTestDurability1"; - HTable importTable = UTIL.createTable(Bytes.toBytes(importTableName), FAMILYA, 3); + Table importTable = UTIL.createTable(Bytes.toBytes(importTableName), FAMILYA, 3); // Register the hlog listener for the import table TableWALActionListener walListener = new TableWALActionListener(importTableName); diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/mapreduce/TestImportTSVWithOperationAttributes.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/mapreduce/TestImportTSVWithOperationAttributes.java index 27f4acb10c4..1c134685b7f 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/mapreduce/TestImportTSVWithOperationAttributes.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/mapreduce/TestImportTSVWithOperationAttributes.java @@ -38,6 +38,7 @@ import org.apache.hadoop.hbase.CellUtil; import org.apache.hadoop.hbase.HBaseTestingUtility; import org.apache.hadoop.hbase.HConstants; import org.apache.hadoop.hbase.LargeTests; +import org.apache.hadoop.hbase.client.Admin; import org.apache.hadoop.hbase.client.Durability; import org.apache.hadoop.hbase.client.HBaseAdmin; import org.apache.hadoop.hbase.client.HTable; @@ -45,6 +46,7 @@ import org.apache.hadoop.hbase.client.Put; import org.apache.hadoop.hbase.client.Result; import org.apache.hadoop.hbase.client.ResultScanner; import org.apache.hadoop.hbase.client.Scan; +import org.apache.hadoop.hbase.client.Table; import org.apache.hadoop.hbase.coprocessor.BaseRegionObserver; import org.apache.hadoop.hbase.coprocessor.ObserverContext; import org.apache.hadoop.hbase.coprocessor.RegionCoprocessorEnvironment; @@ -96,7 +98,7 @@ public class TestImportTSVWithOperationAttributes implements Configurable { conf.set("hbase.coprocessor.master.classes", OperationAttributesTestController.class.getName()); conf.set("hbase.coprocessor.region.classes", OperationAttributesTestController.class.getName()); util.startMiniCluster(); - HBaseAdmin admin = new HBaseAdmin(util.getConfiguration()); + Admin admin = new HBaseAdmin(util.getConfiguration()); util.startMiniMapReduceCluster(); } @@ -192,7 +194,7 @@ public class TestImportTSVWithOperationAttributes implements Configurable { int valueMultiplier, boolean dataAvailable) throws IOException { LOG.debug("Validating table."); - HTable table = new HTable(conf, tableName); + Table table = new HTable(conf, tableName); boolean verified = false; long pause = conf.getLong("hbase.client.pause", 5 * 1000); int numRetries = conf.getInt(HConstants.HBASE_CLIENT_RETRIES_NUMBER, 5); diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/mapreduce/TestImportTSVWithVisibilityLabels.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/mapreduce/TestImportTSVWithVisibilityLabels.java index 0c483aa080c..9494fa489c3 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/mapreduce/TestImportTSVWithVisibilityLabels.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/mapreduce/TestImportTSVWithVisibilityLabels.java @@ -42,12 +42,14 @@ import org.apache.hadoop.hbase.CellUtil; import org.apache.hadoop.hbase.HBaseTestingUtility; import org.apache.hadoop.hbase.HConstants; import org.apache.hadoop.hbase.LargeTests; +import org.apache.hadoop.hbase.client.Admin; import org.apache.hadoop.hbase.client.Delete; import org.apache.hadoop.hbase.client.HBaseAdmin; import org.apache.hadoop.hbase.client.HTable; import org.apache.hadoop.hbase.client.Result; import org.apache.hadoop.hbase.client.ResultScanner; import org.apache.hadoop.hbase.client.Scan; +import org.apache.hadoop.hbase.client.Table; import org.apache.hadoop.hbase.protobuf.generated.VisibilityLabelsProtos.VisibilityLabelsResponse; import org.apache.hadoop.hbase.security.User; import org.apache.hadoop.hbase.security.visibility.Authorizations; @@ -118,7 +120,7 @@ public class TestImportTSVWithVisibilityLabels implements Configurable { // Wait for the labels table to become available util.waitTableEnabled(VisibilityConstants.LABELS_TABLE_NAME.getName(), 50000); createLabels(); - HBaseAdmin admin = new HBaseAdmin(util.getConfiguration()); + Admin admin = new HBaseAdmin(util.getConfiguration()); util.startMiniMapReduceCluster(); } @@ -181,7 +183,7 @@ public class TestImportTSVWithVisibilityLabels implements Configurable { private void issueDeleteAndVerifyData(String tableName) throws IOException { LOG.debug("Validating table after delete."); - HTable table = new HTable(conf, tableName); + Table table = new HTable(conf, tableName); boolean verified = false; long pause = conf.getLong("hbase.client.pause", 5 * 1000); int numRetries = conf.getInt(HConstants.HBASE_CLIENT_RETRIES_NUMBER, 5); @@ -365,7 +367,7 @@ public class TestImportTSVWithVisibilityLabels implements Configurable { int valueMultiplier) throws IOException { LOG.debug("Validating table."); - HTable table = new HTable(conf, tableName); + Table table = new HTable(conf, tableName); boolean verified = false; long pause = conf.getLong("hbase.client.pause", 5 * 1000); int numRetries = conf.getInt(HConstants.HBASE_CLIENT_RETRIES_NUMBER, 5); diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/mapreduce/TestImportTsv.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/mapreduce/TestImportTsv.java index 075c0ab9f14..8e64f77f4b1 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/mapreduce/TestImportTsv.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/mapreduce/TestImportTsv.java @@ -46,6 +46,7 @@ import org.apache.hadoop.hbase.client.HTable; import org.apache.hadoop.hbase.client.Result; import org.apache.hadoop.hbase.client.ResultScanner; import org.apache.hadoop.hbase.client.Scan; +import org.apache.hadoop.hbase.client.Table; import org.apache.hadoop.hbase.util.Bytes; import org.apache.hadoop.io.Text; import org.apache.hadoop.mapred.Utils.OutputFileUtils.OutputFilesFilter; @@ -302,7 +303,7 @@ public class TestImportTsv implements Configurable { String family, int valueMultiplier) throws IOException { LOG.debug("Validating table."); - HTable table = new HTable(conf, tableName); + Table table = new HTable(conf, tableName); boolean verified = false; long pause = conf.getLong("hbase.client.pause", 5 * 1000); int numRetries = conf.getInt(HConstants.HBASE_CLIENT_RETRIES_NUMBER, 5); diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/mapreduce/TestLoadIncrementalHFiles.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/mapreduce/TestLoadIncrementalHFiles.java index 27c809a3b71..2695706f382 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/mapreduce/TestLoadIncrementalHFiles.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/mapreduce/TestLoadIncrementalHFiles.java @@ -35,8 +35,8 @@ import org.apache.hadoop.hbase.HTableDescriptor; import org.apache.hadoop.hbase.LargeTests; import org.apache.hadoop.hbase.NamespaceDescriptor; import org.apache.hadoop.hbase.TableName; -import org.apache.hadoop.hbase.client.HBaseAdmin; import org.apache.hadoop.hbase.client.HTable; +import org.apache.hadoop.hbase.client.Table; import org.apache.hadoop.hbase.io.hfile.CacheConfig; import org.apache.hadoop.hbase.io.hfile.HFile; import org.apache.hadoop.hbase.io.hfile.HFileScanner; @@ -249,7 +249,7 @@ public class TestLoadIncrementalHFiles { String [] args= {dir.toString(), tableName.toString()}; loader.run(args); - HTable table = new HTable(util.getConfiguration(), tableName); + Table table = new HTable(util.getConfiguration(), tableName); try { assertEquals(expectedRows, util.countRows(table)); } finally { diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/mapreduce/TestLoadIncrementalHFilesSplitRecovery.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/mapreduce/TestLoadIncrementalHFilesSplitRecovery.java index 491c2c67d63..c89bd51f3ff 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/mapreduce/TestLoadIncrementalHFilesSplitRecovery.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/mapreduce/TestLoadIncrementalHFilesSplitRecovery.java @@ -52,6 +52,7 @@ import org.apache.hadoop.hbase.client.HTable; import org.apache.hadoop.hbase.client.Result; import org.apache.hadoop.hbase.client.ResultScanner; import org.apache.hadoop.hbase.client.Scan; +import org.apache.hadoop.hbase.client.Table; import org.apache.hadoop.hbase.protobuf.ProtobufUtil; import org.apache.hadoop.hbase.protobuf.generated.ClientProtos; import org.apache.hadoop.hbase.protobuf.generated.ClientProtos.BulkLoadHFileRequest; @@ -229,7 +230,7 @@ public class TestLoadIncrementalHFilesSplitRecovery { * @throws IOException */ void assertExpectedTable(String table, int count, int value) throws IOException { - HTable t = null; + Table t = null; try { assertEquals(util.getHBaseAdmin().listTables(table).length, 1); t = new HTable(util.getConfiguration(), table); @@ -341,7 +342,7 @@ public class TestLoadIncrementalHFilesSplitRecovery { LoadIncrementalHFiles lih2 = new LoadIncrementalHFiles( util.getConfiguration()) { - protected void bulkLoadPhase(final HTable htable, final HConnection conn, + protected void bulkLoadPhase(final Table htable, final HConnection conn, ExecutorService pool, Deque queue, final Multimap regionGroups) throws IOException { int i = attemptedCalls.incrementAndGet(); diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/mapreduce/TestMultithreadedTableMapper.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/mapreduce/TestMultithreadedTableMapper.java index 34bca43eb51..93dbbebc586 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/mapreduce/TestMultithreadedTableMapper.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/mapreduce/TestMultithreadedTableMapper.java @@ -34,6 +34,7 @@ import org.apache.hadoop.hbase.client.Put; import org.apache.hadoop.hbase.client.Result; import org.apache.hadoop.hbase.client.ResultScanner; import org.apache.hadoop.hbase.client.Scan; +import org.apache.hadoop.hbase.client.Table; import org.apache.hadoop.hbase.io.ImmutableBytesWritable; import org.apache.hadoop.hbase.util.Bytes; import org.apache.hadoop.mapreduce.Job; @@ -160,7 +161,7 @@ public class TestMultithreadedTableMapper { } private void verify(String tableName) throws IOException { - HTable table = new HTable(new Configuration(UTIL.getConfiguration()), tableName); + Table table = new HTable(new Configuration(UTIL.getConfiguration()), tableName); boolean verified = false; long pause = UTIL.getConfiguration().getLong("hbase.client.pause", 5 * 1000); int numRetries = UTIL.getConfiguration().getInt(HConstants.HBASE_CLIENT_RETRIES_NUMBER, 5); @@ -193,7 +194,7 @@ public class TestMultithreadedTableMapper { * @throws IOException * @throws NullPointerException if we failed to find a cell value */ - private void verifyAttempt(final HTable table) + private void verifyAttempt(final Table table) throws IOException, NullPointerException { Scan scan = new Scan(); scan.addFamily(INPUT_FAMILY); diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/mapreduce/TestRowCounter.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/mapreduce/TestRowCounter.java index f5bbddcd1b5..09aa652b816 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/mapreduce/TestRowCounter.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/mapreduce/TestRowCounter.java @@ -32,8 +32,8 @@ import org.apache.commons.logging.LogFactory; import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.hbase.HBaseTestingUtility; import org.apache.hadoop.hbase.MediumTests; -import org.apache.hadoop.hbase.client.HTable; import org.apache.hadoop.hbase.client.Put; +import org.apache.hadoop.hbase.client.Table; import org.apache.hadoop.hbase.mapreduce.RowCounter.RowCounterMapper; import org.apache.hadoop.hbase.util.Bytes; import org.apache.hadoop.hbase.util.LauncherSecurityManager; @@ -67,7 +67,7 @@ public class TestRowCounter { public static void setUpBeforeClass() throws Exception { TEST_UTIL.startMiniCluster(); TEST_UTIL.startMiniMapReduceCluster(); - HTable table = TEST_UTIL.createTable(Bytes.toBytes(TABLE_NAME), + Table table = TEST_UTIL.createTable(Bytes.toBytes(TABLE_NAME), Bytes.toBytes(COL_FAM)); writeRows(table); table.close(); @@ -164,7 +164,7 @@ public class TestRowCounter { * @param table * @throws IOException */ - private static void writeRows(HTable table) throws IOException { + private static void writeRows(Table table) throws IOException { final byte[] family = Bytes.toBytes(COL_FAM); final byte[] value = Bytes.toBytes("abcd"); final byte[] col1 = Bytes.toBytes(COL1); diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/mapreduce/TestTableMapReduceBase.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/mapreduce/TestTableMapReduceBase.java index 8a45f14ae31..789c87455bf 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/mapreduce/TestTableMapReduceBase.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/mapreduce/TestTableMapReduceBase.java @@ -37,6 +37,7 @@ import org.apache.hadoop.hbase.client.Put; import org.apache.hadoop.hbase.client.Result; import org.apache.hadoop.hbase.client.ResultScanner; import org.apache.hadoop.hbase.client.Scan; +import org.apache.hadoop.hbase.client.Table; import org.apache.hadoop.hbase.io.ImmutableBytesWritable; import org.apache.hadoop.hbase.util.Bytes; import org.junit.AfterClass; @@ -132,7 +133,7 @@ public abstract class TestTableMapReduceBase { } protected void verify(String tableName) throws IOException { - HTable table = new HTable(UTIL.getConfiguration(), tableName); + Table table = new HTable(UTIL.getConfiguration(), tableName); boolean verified = false; long pause = UTIL.getConfiguration().getLong("hbase.client.pause", 5 * 1000); int numRetries = UTIL.getConfiguration().getInt(HConstants.HBASE_CLIENT_RETRIES_NUMBER, 5); @@ -163,7 +164,7 @@ public abstract class TestTableMapReduceBase { * @throws IOException * @throws NullPointerException if we failed to find a cell value */ - private void verifyAttempt(final HTable table) throws IOException, NullPointerException { + private void verifyAttempt(final Table table) throws IOException, NullPointerException { Scan scan = new Scan(); TableInputFormat.addColumns(scan, columns); ResultScanner scanner = table.getScanner(scan); diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/mapreduce/TestTimeRangeMapRed.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/mapreduce/TestTimeRangeMapRed.java index a5c0b927509..e6520dfc095 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/mapreduce/TestTimeRangeMapRed.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/mapreduce/TestTimeRangeMapRed.java @@ -32,6 +32,7 @@ import org.apache.hadoop.conf.Configurable; import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.fs.FileUtil; import org.apache.hadoop.hbase.*; +import org.apache.hadoop.hbase.client.Admin; import org.apache.hadoop.hbase.client.HBaseAdmin; import org.apache.hadoop.hbase.client.HTable; import org.apache.hadoop.hbase.client.Put; @@ -39,6 +40,7 @@ import org.apache.hadoop.hbase.client.Result; import org.apache.hadoop.hbase.client.ResultScanner; import org.apache.hadoop.hbase.client.Scan; import org.apache.hadoop.hbase.client.Durability; +import org.apache.hadoop.hbase.client.Table; import org.apache.hadoop.hbase.io.ImmutableBytesWritable; import org.apache.hadoop.hbase.util.Bytes; import org.apache.hadoop.io.MapWritable; @@ -57,7 +59,7 @@ public class TestTimeRangeMapRed { private final static Log log = LogFactory.getLog(TestTimeRangeMapRed.class); private static final HBaseTestingUtility UTIL = new HBaseTestingUtility(); - private HBaseAdmin admin; + private Admin admin; private static final byte [] KEY = Bytes.toBytes("row1"); private static final NavigableMap TIMESTAMP = @@ -103,7 +105,7 @@ public class TestTimeRangeMapRed { implements Configurable { private Configuration conf = null; - private HTable table = null; + private Table table = null; @Override public void map(ImmutableBytesWritable key, Result result, @@ -147,13 +149,13 @@ public class TestTimeRangeMapRed { col.setMaxVersions(Integer.MAX_VALUE); desc.addFamily(col); admin.createTable(desc); - HTable table = new HTable(UTIL.getConfiguration(), desc.getTableName()); + Table table = new HTable(UTIL.getConfiguration(), desc.getTableName()); prepareTest(table); runTestOnTable(); verify(table); } - private void prepareTest(final HTable table) throws IOException { + private void prepareTest(final Table table) throws IOException { for (Map.Entry entry : TIMESTAMP.entrySet()) { Put put = new Put(KEY); put.setDurability(Durability.SKIP_WAL); @@ -190,7 +192,7 @@ public class TestTimeRangeMapRed { } } - private void verify(final HTable table) throws IOException { + private void verify(final Table table) throws IOException { Scan scan = new Scan(); scan.addColumn(FAMILY_NAME, COLUMN_NAME); scan.setMaxVersions(1); diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/mapreduce/TestWALPlayer.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/mapreduce/TestWALPlayer.java index 89a46d88339..a586af5713b 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/mapreduce/TestWALPlayer.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/mapreduce/TestWALPlayer.java @@ -41,9 +41,9 @@ import org.apache.hadoop.hbase.MiniHBaseCluster; import org.apache.hadoop.hbase.TableName; import org.apache.hadoop.hbase.client.Delete; import org.apache.hadoop.hbase.client.Get; -import org.apache.hadoop.hbase.client.HTable; import org.apache.hadoop.hbase.client.Put; import org.apache.hadoop.hbase.client.Result; +import org.apache.hadoop.hbase.client.Table; import org.apache.hadoop.hbase.io.ImmutableBytesWritable; import org.apache.hadoop.hbase.mapreduce.WALPlayer.HLogKeyValueMapper; import org.apache.hadoop.hbase.regionserver.wal.HLog; @@ -92,8 +92,8 @@ public class TestWALPlayer { final byte[] COLUMN1 = Bytes.toBytes("c1"); final byte[] COLUMN2 = Bytes.toBytes("c2"); final byte[] ROW = Bytes.toBytes("row"); - HTable t1 = TEST_UTIL.createTable(TABLENAME1, FAMILY); - HTable t2 = TEST_UTIL.createTable(TABLENAME2, FAMILY); + Table t1 = TEST_UTIL.createTable(TABLENAME1, FAMILY); + Table t2 = TEST_UTIL.createTable(TABLENAME2, FAMILY); // put a row into the first table Put p = new Put(ROW); diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestAssignmentListener.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestAssignmentListener.java index 5fb67e5e0bb..383baf80de4 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestAssignmentListener.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestAssignmentListener.java @@ -35,6 +35,7 @@ import org.apache.hadoop.hbase.ServerName; import org.apache.hadoop.hbase.client.Admin; 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.regionserver.HRegion; import org.apache.hadoop.hbase.util.Bytes; import org.apache.hadoop.hbase.util.JVMClusterUtil; @@ -203,7 +204,7 @@ public class TestAssignmentListener { assertEquals(0, listener.getCloseCount()); // Add some data - HTable table = new HTable(TEST_UTIL.getConfiguration(), TABLE_NAME); + Table table = new HTable(TEST_UTIL.getConfiguration(), TABLE_NAME); try { for (int i = 0; i < 10; ++i) { byte[] key = Bytes.toBytes("row-" + i); diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestAssignmentManagerOnCluster.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestAssignmentManagerOnCluster.java index 1f74710dc9b..2208bf2081f 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestAssignmentManagerOnCluster.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestAssignmentManagerOnCluster.java @@ -52,6 +52,7 @@ import org.apache.hadoop.hbase.MetaTableAccessor; import org.apache.hadoop.hbase.client.Admin; import org.apache.hadoop.hbase.client.HTable; import org.apache.hadoop.hbase.client.Result; +import org.apache.hadoop.hbase.client.Table; import org.apache.hadoop.hbase.coordination.ZkCoordinatedStateManager; import org.apache.hadoop.hbase.coprocessor.BaseRegionObserver; import org.apache.hadoop.hbase.coprocessor.CoprocessorHost; @@ -169,7 +170,7 @@ public class TestAssignmentManagerOnCluster { desc.addFamily(new HColumnDescriptor(FAMILY)); admin.createTable(desc); - HTable meta = new HTable(conf, TableName.META_TABLE_NAME); + Table meta = new HTable(conf, TableName.META_TABLE_NAME); HRegionInfo hri = new HRegionInfo( desc.getTableName(), Bytes.toBytes("A"), Bytes.toBytes("Z")); MetaTableAccessor.addRegionToMeta(meta, hri); @@ -213,7 +214,7 @@ public class TestAssignmentManagerOnCluster { desc.addFamily(new HColumnDescriptor(FAMILY)); admin.createTable(desc); - HTable meta = new HTable(conf, TableName.META_TABLE_NAME); + Table meta = new HTable(conf, TableName.META_TABLE_NAME); final HRegionInfo hri = new HRegionInfo( desc.getTableName(), Bytes.toBytes("A"), Bytes.toBytes("Z")); MetaTableAccessor.addRegionToMeta(meta, hri); @@ -427,7 +428,7 @@ public class TestAssignmentManagerOnCluster { desc.addFamily(new HColumnDescriptor(FAMILY)); admin.createTable(desc); - HTable meta = new HTable(conf, TableName.META_TABLE_NAME); + Table meta = new HTable(conf, TableName.META_TABLE_NAME); HRegionInfo hri = new HRegionInfo( desc.getTableName(), Bytes.toBytes("A"), Bytes.toBytes("Z")); MetaTableAccessor.addRegionToMeta(meta, hri); @@ -476,7 +477,7 @@ public class TestAssignmentManagerOnCluster { desc.addFamily(new HColumnDescriptor(FAMILY)); admin.createTable(desc); - HTable meta = new HTable(conf, TableName.META_TABLE_NAME); + Table meta = new HTable(conf, TableName.META_TABLE_NAME); HRegionInfo hri = new HRegionInfo( desc.getTableName(), Bytes.toBytes("A"), Bytes.toBytes("Z")); MetaTableAccessor.addRegionToMeta(meta, hri); @@ -522,7 +523,7 @@ public class TestAssignmentManagerOnCluster { desc.addFamily(new HColumnDescriptor(FAMILY)); admin.createTable(desc); - HTable meta = new HTable(conf, TableName.META_TABLE_NAME); + Table meta = new HTable(conf, TableName.META_TABLE_NAME); HRegionInfo hri = new HRegionInfo( desc.getTableName(), Bytes.toBytes("A"), Bytes.toBytes("Z")); MetaTableAccessor.addRegionToMeta(meta, hri); @@ -564,7 +565,7 @@ public class TestAssignmentManagerOnCluster { desc.addFamily(new HColumnDescriptor(FAMILY)); admin.createTable(desc); - HTable meta = new HTable(conf, TableName.META_TABLE_NAME); + Table meta = new HTable(conf, TableName.META_TABLE_NAME); HRegionInfo hri = new HRegionInfo( desc.getTableName(), Bytes.toBytes("A"), Bytes.toBytes("Z")); MetaTableAccessor.addRegionToMeta(meta, hri); @@ -662,7 +663,7 @@ public class TestAssignmentManagerOnCluster { desc.addFamily(new HColumnDescriptor(FAMILY)); admin.createTable(desc); - HTable meta = new HTable(conf, TableName.META_TABLE_NAME); + Table meta = new HTable(conf, TableName.META_TABLE_NAME); HRegionInfo hri = new HRegionInfo( desc.getTableName(), Bytes.toBytes("A"), Bytes.toBytes("Z")); MetaTableAccessor.addRegionToMeta(meta, hri); @@ -716,7 +717,7 @@ public class TestAssignmentManagerOnCluster { desc.addFamily(new HColumnDescriptor(FAMILY)); admin.createTable(desc); - HTable meta = new HTable(conf, TableName.META_TABLE_NAME); + Table meta = new HTable(conf, TableName.META_TABLE_NAME); HRegionInfo hri = new HRegionInfo( desc.getTableName(), Bytes.toBytes("A"), Bytes.toBytes("Z")); MetaTableAccessor.addRegionToMeta(meta, hri); @@ -789,7 +790,7 @@ public class TestAssignmentManagerOnCluster { desc.addFamily(new HColumnDescriptor(FAMILY)); admin.createTable(desc); - HTable meta = new HTable(conf, TableName.META_TABLE_NAME); + Table meta = new HTable(conf, TableName.META_TABLE_NAME); HRegionInfo hri = new HRegionInfo( desc.getTableName(), Bytes.toBytes("A"), Bytes.toBytes("Z")); MetaTableAccessor.addRegionToMeta(meta, hri); @@ -864,7 +865,7 @@ public class TestAssignmentManagerOnCluster { desc.addFamily(new HColumnDescriptor(FAMILY)); admin.createTable(desc); - HTable meta = new HTable(conf, TableName.META_TABLE_NAME); + Table meta = new HTable(conf, TableName.META_TABLE_NAME); HRegionInfo hri = new HRegionInfo( desc.getTableName(), Bytes.toBytes("A"), Bytes.toBytes("Z")); MetaTableAccessor.addRegionToMeta(meta, hri); @@ -905,7 +906,7 @@ public class TestAssignmentManagerOnCluster { desc.addFamily(new HColumnDescriptor(FAMILY)); admin.createTable(desc); - HTable meta = new HTable(conf, TableName.META_TABLE_NAME); + Table meta = new HTable(conf, TableName.META_TABLE_NAME); HRegionInfo hri = new HRegionInfo( desc.getTableName(), Bytes.toBytes("A"), Bytes.toBytes("Z")); MetaTableAccessor.addRegionToMeta(meta, hri); @@ -984,7 +985,7 @@ public class TestAssignmentManagerOnCluster { desc.addFamily(new HColumnDescriptor(FAMILY)); admin.createTable(desc); - HTable meta = new HTable(conf, TableName.META_TABLE_NAME); + Table meta = new HTable(conf, TableName.META_TABLE_NAME); HRegionInfo hri = new HRegionInfo( desc.getTableName(), Bytes.toBytes("A"), Bytes.toBytes("Z")); MetaTableAccessor.addRegionToMeta(meta, hri); @@ -1059,7 +1060,7 @@ public class TestAssignmentManagerOnCluster { HTableDescriptor desc = new HTableDescriptor(TableName.valueOf(table)); desc.addFamily(new HColumnDescriptor(FAMILY)); admin.createTable(desc); - HTable meta = new HTable(conf, TableName.META_TABLE_NAME); + Table meta = new HTable(conf, TableName.META_TABLE_NAME); HRegionInfo hri = new HRegionInfo(desc.getTableName(), Bytes.toBytes("A"), Bytes.toBytes("Z")); MetaTableAccessor.addRegionToMeta(meta, hri); diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestDistributedLogSplitting.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestDistributedLogSplitting.java index 390cd47b04c..b5400b2c7e7 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestDistributedLogSplitting.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestDistributedLogSplitting.java @@ -77,6 +77,7 @@ import org.apache.hadoop.hbase.client.PerClientRandomNonceGenerator; import org.apache.hadoop.hbase.client.Put; import org.apache.hadoop.hbase.client.Result; import org.apache.hadoop.hbase.client.RetriesExhaustedWithDetailsException; +import org.apache.hadoop.hbase.client.Table; import org.apache.hadoop.hbase.coordination.BaseCoordinatedStateManager; import org.apache.hadoop.hbase.coordination.SplitLogManagerCoordination; import org.apache.hadoop.hbase.coordination.ZKSplitLogManagerCoordination; @@ -283,7 +284,7 @@ public class TestDistributedLogSplitting { master.balanceSwitch(false); final ZooKeeperWatcher zkw = new ZooKeeperWatcher(conf, "table-creation", null); - HTable ht = installTable(zkw, "table", "family", NUM_REGIONS_TO_CREATE); + Table ht = installTable(zkw, "table", "family", NUM_REGIONS_TO_CREATE); HRegionServer hrs = findRSToKill(false, "table"); List regions = ProtobufUtil.getOnlineRegions(hrs.getRSRpcServices()); @@ -381,7 +382,7 @@ public class TestDistributedLogSplitting { master.balanceSwitch(false); final ZooKeeperWatcher zkw = new ZooKeeperWatcher(conf, "table-creation", null); - HTable ht = installTable(zkw, "table", "family", NUM_REGIONS_TO_CREATE); + Table ht = installTable(zkw, "table", "family", NUM_REGIONS_TO_CREATE); HRegionServer hrs = findRSToKill(true, "table"); List regions = ProtobufUtil.getOnlineRegions(hrs.getRSRpcServices()); @@ -392,7 +393,7 @@ public class TestDistributedLogSplitting { zkw.close(); } - private void abortRSAndVerifyRecovery(HRegionServer hrs, HTable ht, final ZooKeeperWatcher zkw, + private void abortRSAndVerifyRecovery(HRegionServer hrs, Table ht, final ZooKeeperWatcher zkw, final int numRegions, final int numofLines) throws Exception { abortRSAndWaitForRecovery(hrs, zkw, numRegions); @@ -449,7 +450,7 @@ public class TestDistributedLogSplitting { master.balanceSwitch(false); final ZooKeeperWatcher zkw = new ZooKeeperWatcher(conf, "table-creation", null); - HTable ht = installTable(zkw, "table", "family", NUM_REGIONS_TO_CREATE); + Table ht = installTable(zkw, "table", "family", NUM_REGIONS_TO_CREATE); HRegionServer hrs = findRSToKill(false, "table"); List regions = ProtobufUtil.getOnlineRegions(hrs.getRSRpcServices()); @@ -506,7 +507,7 @@ public class TestDistributedLogSplitting { master.balanceSwitch(false); final ZooKeeperWatcher zkw = new ZooKeeperWatcher(conf, "table-creation", null); - HTable ht = installTable(zkw, "table", "family", NUM_REGIONS_TO_CREATE); + Table ht = installTable(zkw, "table", "family", NUM_REGIONS_TO_CREATE); HRegionServer hrs = findRSToKill(false, "table"); List regions = ProtobufUtil.getOnlineRegions(hrs.getRSRpcServices()); @@ -564,7 +565,7 @@ public class TestDistributedLogSplitting { List rsts = cluster.getLiveRegionServerThreads(); final ZooKeeperWatcher zkw = new ZooKeeperWatcher(conf, "table-creation", null); - HTable ht = installTable(zkw, "table", "family", NUM_REGIONS_TO_CREATE); + Table ht = installTable(zkw, "table", "family", NUM_REGIONS_TO_CREATE); List regions = null; HRegionServer hrs1 = findRSToKill(false, "table"); @@ -641,7 +642,7 @@ public class TestDistributedLogSplitting { master.balanceSwitch(false); List rsts = cluster.getLiveRegionServerThreads(); final ZooKeeperWatcher zkw = master.getZooKeeper(); - HTable ht = installTable(zkw, "table", "family", 40); + Table ht = installTable(zkw, "table", "family", 40); final SplitLogManager slm = master.getMasterFileSystem().splitLogManager; Set regionSet = new HashSet(); @@ -734,8 +735,8 @@ public class TestDistributedLogSplitting { List rsts = cluster.getLiveRegionServerThreads(); final ZooKeeperWatcher zkw = new ZooKeeperWatcher(conf, "table-creation", null); - HTable disablingHT = installTable(zkw, "disableTable", "family", NUM_REGIONS_TO_CREATE); - HTable ht = installTable(zkw, "table", "family", NUM_REGIONS_TO_CREATE, NUM_REGIONS_TO_CREATE); + Table disablingHT = installTable(zkw, "disableTable", "family", NUM_REGIONS_TO_CREATE); + Table ht = installTable(zkw, "table", "family", NUM_REGIONS_TO_CREATE, NUM_REGIONS_TO_CREATE); // turn off load balancing to prevent regions from moving around otherwise // they will consume recovered.edits @@ -1012,7 +1013,7 @@ public class TestDistributedLogSplitting { final ZooKeeperWatcher zkw = new ZooKeeperWatcher(conf, "distributed log splitting test", null); - HTable ht = installTable(zkw, "table", "family", NUM_REGIONS_TO_CREATE); + Table ht = installTable(zkw, "table", "family", NUM_REGIONS_TO_CREATE); populateDataInTable(NUM_ROWS_PER_REGION, "family"); @@ -1189,7 +1190,7 @@ public class TestDistributedLogSplitting { List rsts = cluster.getLiveRegionServerThreads(); final ZooKeeperWatcher zkw = new ZooKeeperWatcher(conf, "table-creation", null); - HTable ht = installTable(zkw, "table", "family", NUM_REGIONS_TO_CREATE); + Table ht = installTable(zkw, "table", "family", NUM_REGIONS_TO_CREATE); List regions = null; HRegionServer hrs = null; @@ -1281,7 +1282,7 @@ public class TestDistributedLogSplitting { List rsts = cluster.getLiveRegionServerThreads(); final ZooKeeperWatcher zkw = new ZooKeeperWatcher(conf, "table-creation", null); - HTable ht = installTable(zkw, "table", "family", NUM_REGIONS_TO_CREATE); + Table ht = installTable(zkw, "table", "family", NUM_REGIONS_TO_CREATE); List regions = null; HRegionServer hrs = null; diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestMasterFailover.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestMasterFailover.java index 5f1159a10e8..48e2f6f8908 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestMasterFailover.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestMasterFailover.java @@ -53,6 +53,7 @@ import org.apache.hadoop.hbase.MetaTableAccessor; import org.apache.hadoop.hbase.coordination.BaseCoordinatedStateManager; import org.apache.hadoop.hbase.client.HTable; import org.apache.hadoop.hbase.executor.EventType; +import org.apache.hadoop.hbase.client.RegionLocator; import org.apache.hadoop.hbase.master.RegionState.State; import org.apache.hadoop.hbase.protobuf.ProtobufUtil; import org.apache.hadoop.hbase.protobuf.generated.ZooKeeperProtos; diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestMasterOperationsForRegionReplicas.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestMasterOperationsForRegionReplicas.java index 117c51504b8..5f409fe1a8f 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestMasterOperationsForRegionReplicas.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestMasterOperationsForRegionReplicas.java @@ -44,12 +44,14 @@ import org.apache.hadoop.hbase.ServerName; import org.apache.hadoop.hbase.TableName; import org.apache.hadoop.hbase.MetaTableAccessor; import org.apache.hadoop.hbase.MetaTableAccessor.Visitor; +import org.apache.hadoop.hbase.client.Admin; import org.apache.hadoop.hbase.client.Delete; import org.apache.hadoop.hbase.client.HBaseAdmin; import org.apache.hadoop.hbase.client.HConnection; import org.apache.hadoop.hbase.client.HTable; import org.apache.hadoop.hbase.client.RegionReplicaUtil; import org.apache.hadoop.hbase.client.Result; +import org.apache.hadoop.hbase.client.Table; import org.apache.hadoop.hbase.util.Bytes; import org.junit.AfterClass; import org.junit.BeforeClass; @@ -60,7 +62,7 @@ import org.junit.experimental.categories.Category; public class TestMasterOperationsForRegionReplicas { final static Log LOG = LogFactory.getLog(TestRegionPlacement.class); private final static HBaseTestingUtility TEST_UTIL = new HBaseTestingUtility(); - private static HBaseAdmin admin; + private static Admin admin; private static int numSlaves = 2; private static Configuration conf; @@ -252,7 +254,7 @@ public class TestMasterOperationsForRegionReplicas { admin.disableTable(table); // now delete one replica info from all the rows // this is to make the meta appear to be only partially updated - HTable metaTable = new HTable(TableName.META_TABLE_NAME, admin.getConnection()); + Table metaTable = new HTable(TableName.META_TABLE_NAME, admin.getConnection()); for (byte[] row : tableRows) { Delete deleteOneReplicaLocation = new Delete(row); deleteOneReplicaLocation.deleteColumns(HConstants.CATALOG_FAMILY, diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestMasterRestartAfterDisablingTable.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestMasterRestartAfterDisablingTable.java index 7ed455ed3dd..9799ee69f10 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestMasterRestartAfterDisablingTable.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestMasterRestartAfterDisablingTable.java @@ -32,6 +32,7 @@ import org.apache.hadoop.hbase.HBaseTestingUtility; import org.apache.hadoop.hbase.LargeTests; import org.apache.hadoop.hbase.MiniHBaseCluster; import org.apache.hadoop.hbase.TableName; +import org.apache.hadoop.hbase.client.Admin; import org.apache.hadoop.hbase.client.HBaseAdmin; import org.apache.hadoop.hbase.client.HTable; import org.apache.hadoop.hbase.protobuf.generated.ZooKeeperProtos; @@ -102,7 +103,7 @@ public class TestMasterRestartAfterDisablingTable { ZooKeeperProtos.Table.State.DISABLING)); log("Enabling table\n"); // Need a new Admin, the previous one is on the old master - HBaseAdmin admin = new HBaseAdmin(TEST_UTIL.getConfiguration()); + Admin admin = new HBaseAdmin(TEST_UTIL.getConfiguration()); admin.enableTable(table); admin.close(); log("Waiting for no more RIT\n"); diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestMasterTransitions.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestMasterTransitions.java index c7d95adb27d..44b9573701e 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestMasterTransitions.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestMasterTransitions.java @@ -33,6 +33,7 @@ import org.apache.hadoop.hbase.client.Result; import org.apache.hadoop.hbase.client.ResultScanner; import org.apache.hadoop.hbase.client.Scan; import org.apache.hadoop.hbase.client.Durability; +import org.apache.hadoop.hbase.client.Table; import org.apache.hadoop.hbase.util.Bytes; import org.junit.AfterClass; import org.junit.Assert; @@ -479,8 +480,8 @@ public class TestMasterTransitions { * @throws IOException */ private static int addToEachStartKey(final int expected) throws IOException { - HTable t = new HTable(TEST_UTIL.getConfiguration(), TABLENAME); - HTable meta = new HTable(TEST_UTIL.getConfiguration(), + Table t = new HTable(TEST_UTIL.getConfiguration(), TABLENAME); + Table meta = new HTable(TEST_UTIL.getConfiguration(), TableName.META_TABLE_NAME); int rows = 0; Scan scan = new Scan(); diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestRegionPlacement.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestRegionPlacement.java index 5c797b2dc94..7062473379d 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestRegionPlacement.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestRegionPlacement.java @@ -38,7 +38,6 @@ import java.util.concurrent.atomic.AtomicInteger; import org.apache.commons.logging.Log; import org.apache.commons.logging.LogFactory; import org.apache.hadoop.conf.Configuration; -import org.apache.hadoop.hbase.ClusterStatus; import org.apache.hadoop.hbase.HBaseIOException; import org.apache.hadoop.hbase.HBaseTestingUtility; import org.apache.hadoop.hbase.HColumnDescriptor; @@ -50,6 +49,7 @@ import org.apache.hadoop.hbase.MiniHBaseCluster; import org.apache.hadoop.hbase.NamespaceDescriptor; import org.apache.hadoop.hbase.ServerName; import org.apache.hadoop.hbase.TableName; +import org.apache.hadoop.hbase.client.Admin; import org.apache.hadoop.hbase.client.HBaseAdmin; import org.apache.hadoop.hbase.client.HTable; import org.apache.hadoop.hbase.client.MetaScanner; @@ -76,7 +76,7 @@ public class TestRegionPlacement { final static Log LOG = LogFactory.getLog(TestRegionPlacement.class); private final static HBaseTestingUtility TEST_UTIL = new HBaseTestingUtility(); private final static int SLAVES = 10; - private static HBaseAdmin admin; + private static Admin admin; private static RegionPlacementMaintainer rp; private static Position[] positions = Position.values(); private int lastRegionOnPrimaryRSCount = 0; diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestCompactionState.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestCompactionState.java index 807751af909..b55654f21fb 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestCompactionState.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestCompactionState.java @@ -32,8 +32,8 @@ import org.apache.hadoop.hbase.TableName; import org.apache.hadoop.hbase.HBaseTestingUtility; import org.apache.hadoop.hbase.LargeTests; import org.apache.hadoop.hbase.client.HBaseAdmin; -import org.apache.hadoop.hbase.client.HTable; import org.apache.hadoop.hbase.client.Put; +import org.apache.hadoop.hbase.client.Table; import org.apache.hadoop.hbase.protobuf.generated.AdminProtos.GetRegionInfoResponse.CompactionState; import org.apache.hadoop.hbase.util.Bytes; import org.junit.AfterClass; @@ -85,7 +85,7 @@ public class TestCompactionState { byte [] fakecf = Bytes.toBytes("fakecf"); boolean caughtMinorCompact = false; boolean caughtMajorCompact = false; - HTable ht = null; + Table ht = null; try { ht = TEST_UTIL.createTable(table, family); HBaseAdmin admin = new HBaseAdmin(TEST_UTIL.getConfiguration()); @@ -128,7 +128,7 @@ public class TestCompactionState { byte [] family = Bytes.toBytes("family"); byte [][] families = {family, Bytes.add(family, Bytes.toBytes("2")), Bytes.add(family, Bytes.toBytes("3"))}; - HTable ht = null; + Table ht = null; try { ht = TEST_UTIL.createTable(table, families); loadData(ht, families, 3000, flushes); @@ -213,7 +213,7 @@ public class TestCompactionState { return count; } - private static void loadData(final HTable ht, final byte[][] families, + private static void loadData(final Table ht, final byte[][] families, final int rows, final int flushes) throws IOException { List puts = new ArrayList(rows); byte[] qualifier = Bytes.toBytes("val"); diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestEncryptionKeyRotation.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestEncryptionKeyRotation.java index c4049dd744c..4cbbfccc3b2 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestEncryptionKeyRotation.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestEncryptionKeyRotation.java @@ -39,6 +39,7 @@ import org.apache.hadoop.hbase.TableName; import org.apache.hadoop.hbase.Waiter.Predicate; import org.apache.hadoop.hbase.client.HTable; import org.apache.hadoop.hbase.client.Put; +import org.apache.hadoop.hbase.client.Table; import org.apache.hadoop.hbase.io.crypto.Encryption; import org.apache.hadoop.hbase.io.crypto.KeyProviderForTesting; import org.apache.hadoop.hbase.io.crypto.aes.AES; @@ -206,7 +207,7 @@ public class TestEncryptionKeyRotation { TEST_UTIL.getHBaseAdmin().createTable(htd); TEST_UTIL.waitTableAvailable(htd.getName(), 5000); // Create a store file - HTable table = new HTable(conf, htd.getName()); + Table table = new HTable(conf, htd.getName()); try { table.put(new Put(Bytes.toBytes("testrow")) .add(hcd.getName(), Bytes.toBytes("q"), Bytes.toBytes("value"))); diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestEncryptionRandomKeying.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestEncryptionRandomKeying.java index 9e5a7b0cc41..71784be06fe 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestEncryptionRandomKeying.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestEncryptionRandomKeying.java @@ -33,6 +33,7 @@ import org.apache.hadoop.hbase.MediumTests; import org.apache.hadoop.hbase.TableName; import org.apache.hadoop.hbase.client.HTable; import org.apache.hadoop.hbase.client.Put; +import org.apache.hadoop.hbase.client.Table; import org.apache.hadoop.hbase.io.crypto.Encryption; import org.apache.hadoop.hbase.io.crypto.KeyProviderForTesting; import org.apache.hadoop.hbase.io.hfile.CacheConfig; @@ -101,7 +102,7 @@ public class TestEncryptionRandomKeying { TEST_UTIL.waitTableAvailable(htd.getName(), 5000); // Create a store file - HTable table = new HTable(conf, htd.getName()); + Table table = new HTable(conf, htd.getName()); try { table.put(new Put(Bytes.toBytes("testrow")) .add(hcd.getName(), Bytes.toBytes("q"), Bytes.toBytes("value"))); diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestEndToEndSplitTransaction.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestEndToEndSplitTransaction.java index 5189fc8c36b..bca4feec75d 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestEndToEndSplitTransaction.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestEndToEndSplitTransaction.java @@ -53,6 +53,8 @@ import org.apache.hadoop.hbase.client.Put; import org.apache.hadoop.hbase.client.Result; import org.apache.hadoop.hbase.client.Scan; import org.apache.hadoop.hbase.coordination.BaseCoordinatedStateManager; +import org.apache.hadoop.hbase.client.Table; +import org.apache.hadoop.hbase.ipc.PayloadCarryingRpcController; import org.apache.hadoop.hbase.protobuf.ProtobufUtil; import org.apache.hadoop.hbase.protobuf.RequestConverter; import org.apache.hadoop.hbase.protobuf.generated.ClientProtos; @@ -204,7 +206,7 @@ public class TestEndToEndSplitTransaction { //SplitTransaction will update the meta table by offlining the parent region, and adding info //for daughters. - HTable table = TEST_UTIL.createTable(TABLENAME, FAMILY); + Table table = TEST_UTIL.createTable(TABLENAME, FAMILY); Stoppable stopper = new StoppableImplementation(); RegionSplitter regionSplitter = new RegionSplitter(table); @@ -231,13 +233,13 @@ public class TestEndToEndSplitTransaction { static class RegionSplitter extends Thread { Throwable ex; - HTable table; + Table table; TableName tableName; byte[] family; Admin admin; HRegionServer rs; - RegionSplitter(HTable table) throws IOException { + RegionSplitter(Table table) throws IOException { this.table = table; this.tableName = table.getName(); this.family = table.getTableDescriptor().getFamiliesKeys().iterator().next(); @@ -439,7 +441,7 @@ public class TestEndToEndSplitTransaction { long start = System.currentTimeMillis(); log("blocking until region is split:" + Bytes.toStringBinary(regionName)); HRegionInfo daughterA = null, daughterB = null; - HTable metaTable = new HTable(conf, TableName.META_TABLE_NAME); + Table metaTable = new HTable(conf, TableName.META_TABLE_NAME); try { while (System.currentTimeMillis() - start < timeout) { @@ -478,12 +480,12 @@ public class TestEndToEndSplitTransaction { } } - public static Result getRegionRow(HTable metaTable, byte[] regionName) throws IOException { + public static Result getRegionRow(Table metaTable, byte[] regionName) throws IOException { Get get = new Get(regionName); return metaTable.get(get); } - public static void blockUntilRegionIsInMeta(HTable metaTable, long timeout, HRegionInfo hri) + public static void blockUntilRegionIsInMeta(Table metaTable, long timeout, HRegionInfo hri) throws IOException, InterruptedException { log("blocking until region is in META: " + hri.getRegionNameAsString()); long start = System.currentTimeMillis(); @@ -504,7 +506,7 @@ public class TestEndToEndSplitTransaction { throws IOException, InterruptedException { log("blocking until region is opened for reading:" + hri.getRegionNameAsString()); long start = System.currentTimeMillis(); - HTable table = new HTable(conf, hri.getTable()); + Table table = new HTable(conf, hri.getTable()); try { byte [] row = hri.getStartKey(); diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestFSErrorsExposed.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestFSErrorsExposed.java index 9c9ec70179e..4884a812f98 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestFSErrorsExposed.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestFSErrorsExposed.java @@ -44,6 +44,7 @@ import org.apache.hadoop.hbase.HTableDescriptor; import org.apache.hadoop.hbase.KeyValue; import org.apache.hadoop.hbase.MediumTests; import org.apache.hadoop.hbase.TableName; +import org.apache.hadoop.hbase.client.Admin; import org.apache.hadoop.hbase.client.HBaseAdmin; import org.apache.hadoop.hbase.client.HTable; import org.apache.hadoop.hbase.fs.HFileSystem; @@ -188,7 +189,7 @@ public class TestFSErrorsExposed { byte[] tableName = Bytes.toBytes("table"); byte[] fam = Bytes.toBytes("fam"); - HBaseAdmin admin = new HBaseAdmin(util.getConfiguration()); + Admin admin = new HBaseAdmin(util.getConfiguration()); HTableDescriptor desc = new HTableDescriptor(TableName.valueOf(tableName)); desc.addFamily(new HColumnDescriptor(fam) .setMaxVersions(1) diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestHRegion.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestHRegion.java index d3d97cab976..99b7a663c64 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestHRegion.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestHRegion.java @@ -95,11 +95,11 @@ import org.apache.hadoop.hbase.client.Append; import org.apache.hadoop.hbase.client.Delete; import org.apache.hadoop.hbase.client.Durability; import org.apache.hadoop.hbase.client.Get; -import org.apache.hadoop.hbase.client.HTable; import org.apache.hadoop.hbase.client.Increment; import org.apache.hadoop.hbase.client.Put; import org.apache.hadoop.hbase.client.Result; import org.apache.hadoop.hbase.client.Scan; +import org.apache.hadoop.hbase.client.Table; import org.apache.hadoop.hbase.exceptions.FailedSanityCheckException; import org.apache.hadoop.hbase.filter.BinaryComparator; import org.apache.hadoop.hbase.filter.ColumnCountGetFilter; @@ -4045,7 +4045,7 @@ public class TestHRegion { try { cluster = htu.startMiniCluster(1, regionServersCount, dataNodeHosts); byte[][] families = { fam1, fam2 }; - HTable ht = htu.createTable(Bytes.toBytes(this.getName()), families); + Table ht = htu.createTable(Bytes.toBytes(this.getName()), families); // Setting up region byte row[] = Bytes.toBytes("row1"); diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestHRegionOnCluster.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestHRegionOnCluster.java index 2c4ac95dd6e..d0ebf8a4e8c 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestHRegionOnCluster.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestHRegionOnCluster.java @@ -30,12 +30,12 @@ import org.apache.hadoop.hbase.HTableDescriptor; import org.apache.hadoop.hbase.MiniHBaseCluster; import org.apache.hadoop.hbase.TableName; import org.apache.hadoop.hbase.client.Admin; -import org.apache.hadoop.hbase.client.HBaseAdmin; import org.apache.hadoop.hbase.client.HTable; import org.apache.hadoop.hbase.client.Put; import org.apache.hadoop.hbase.client.Result; import org.apache.hadoop.hbase.client.ResultScanner; import org.apache.hadoop.hbase.client.Scan; +import org.apache.hadoop.hbase.client.Table; import org.apache.hadoop.hbase.master.HMaster; import org.apache.hadoop.hbase.util.Bytes; import org.apache.hadoop.hbase.MediumTests; @@ -133,7 +133,7 @@ public class TestHRegionOnCluster { } } - private void putDataAndVerify(HTable table, String row, byte[] family, + private void putDataAndVerify(Table table, String row, byte[] family, String value, int verifyNum) throws IOException { System.out.println("=========Putting data :" + row); Put put = new Put(Bytes.toBytes(row)); diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestJoinedScanners.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestJoinedScanners.java index 8f485cbb127..e3c5d3a0446 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestJoinedScanners.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestJoinedScanners.java @@ -46,6 +46,7 @@ import org.apache.hadoop.hbase.client.Put; import org.apache.hadoop.hbase.client.Result; import org.apache.hadoop.hbase.client.ResultScanner; import org.apache.hadoop.hbase.client.Scan; +import org.apache.hadoop.hbase.client.Table; import org.apache.hadoop.hbase.filter.CompareFilter; import org.apache.hadoop.hbase.filter.SingleColumnValueFilter; import org.apache.hadoop.hbase.io.encoding.DataBlockEncoding; @@ -100,7 +101,7 @@ public class TestJoinedScanners { desc.addFamily(hcd); } htu.getHBaseAdmin().createTable(desc); - HTable ht = new HTable(htu.getConfiguration(), tableName); + Table ht = new HTable(htu.getConfiguration(), tableName); long rows_to_insert = 1000; int insert_batch = 20; @@ -150,7 +151,7 @@ public class TestJoinedScanners { } } - private void runScanner(HTable table, boolean slow) throws Exception { + private void runScanner(Table table, boolean slow) throws Exception { long time = System.nanoTime(); Scan scan = new Scan(); scan.addColumn(cf_essential, col_name); diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestRegionMergeTransactionOnCluster.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestRegionMergeTransactionOnCluster.java index 32704115f2b..3352db13e19 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestRegionMergeTransactionOnCluster.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestRegionMergeTransactionOnCluster.java @@ -48,6 +48,7 @@ import org.apache.hadoop.hbase.client.Put; import org.apache.hadoop.hbase.client.Result; import org.apache.hadoop.hbase.client.ResultScanner; import org.apache.hadoop.hbase.client.Scan; +import org.apache.hadoop.hbase.client.Table; import org.apache.hadoop.hbase.exceptions.MergeRegionException; import org.apache.hadoop.hbase.master.AssignmentManager; import org.apache.hadoop.hbase.master.HMaster; @@ -120,7 +121,7 @@ public class TestRegionMergeTransactionOnCluster { TableName.valueOf("testWholesomeMerge"); // Create table and load data. - HTable table = createTableAndLoadData(master, tableName); + Table table = createTableAndLoadData(master, tableName); // Merge 1st and 2nd region mergeRegionsAndVerifyRegionNum(master, tableName, 0, 1, INITIAL_REGION_NUM - 1); @@ -168,7 +169,7 @@ public class TestRegionMergeTransactionOnCluster { final TableName tableName = TableName.valueOf("testCleanMergeReference"); // Create table and load data. - HTable table = createTableAndLoadData(master, tableName); + Table table = createTableAndLoadData(master, tableName); // Merge 1st and 2nd region mergeRegionsAndVerifyRegionNum(master, tableName, 0, 1, INITIAL_REGION_NUM - 1); @@ -249,7 +250,7 @@ public class TestRegionMergeTransactionOnCluster { try { // Create table and load data. - HTable table = createTableAndLoadData(master, tableName); + Table table = createTableAndLoadData(master, tableName); RegionStates regionStates = master.getAssignmentManager().getRegionStates(); List regions = regionStates.getRegionsOfTable(tableName); // Fake offline one region @@ -334,12 +335,12 @@ public class TestRegionMergeTransactionOnCluster { assertEquals(expectedRegionNum, tableRegionsInMeta.size()); } - private HTable createTableAndLoadData(HMaster master, TableName tablename) + private Table createTableAndLoadData(HMaster master, TableName tablename) throws Exception { return createTableAndLoadData(master, tablename, INITIAL_REGION_NUM); } - private HTable createTableAndLoadData(HMaster master, TableName tablename, + private Table createTableAndLoadData(HMaster master, TableName tablename, int numRegions) throws Exception { assertTrue("ROWSIZE must > numregions:" + numRegions, ROWSIZE > numRegions); byte[][] splitRows = new byte[numRegions - 1][]; @@ -347,7 +348,7 @@ public class TestRegionMergeTransactionOnCluster { splitRows[i] = ROWS[(i + 1) * ROWSIZE / numRegions]; } - HTable table = TEST_UTIL.createTable(tablename, FAMILYNAME, splitRows); + Table table = TEST_UTIL.createTable(tablename, FAMILYNAME, splitRows); loadData(table); verifyRowCount(table, ROWSIZE); @@ -377,7 +378,7 @@ public class TestRegionMergeTransactionOnCluster { return ret; } - private void loadData(HTable table) throws IOException { + private void loadData(Table table) throws IOException { for (int i = 0; i < ROWSIZE; i++) { Put put = new Put(ROWS[i]); put.add(FAMILYNAME, QUALIFIER, Bytes.toBytes(i)); @@ -385,7 +386,7 @@ public class TestRegionMergeTransactionOnCluster { } } - private void verifyRowCount(HTable table, int expectedRegionNum) + private void verifyRowCount(Table table, int expectedRegionNum) throws IOException { ResultScanner scanner = table.getScanner(new Scan()); int rowCount = 0; diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestRegionReplicas.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestRegionReplicas.java index 50542be7032..b3bfaaad5a4 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestRegionReplicas.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestRegionReplicas.java @@ -40,6 +40,7 @@ import org.apache.hadoop.hbase.client.Get; import org.apache.hadoop.hbase.client.HTable; import org.apache.hadoop.hbase.client.Put; import org.apache.hadoop.hbase.client.Result; +import org.apache.hadoop.hbase.client.Table; import org.apache.hadoop.hbase.protobuf.ProtobufUtil; import org.apache.hadoop.hbase.protobuf.RequestConverter; import org.apache.hadoop.hbase.protobuf.generated.AdminProtos; @@ -182,7 +183,7 @@ public class TestRegionReplicas { @Test(timeout = 60000) public void testRegionReplicaUpdatesMetaLocation() throws Exception { openRegion(hriSecondary); - HTable meta = null; + Table meta = null; try { meta = new HTable(HTU.getConfiguration(), TableName.META_TABLE_NAME); TestMetaTableAccessor.assertMetaLocation(meta, hriPrimary.getRegionName() diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestRegionServerMetrics.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestRegionServerMetrics.java index aa354864c8a..7ae0b5b5321 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestRegionServerMetrics.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestRegionServerMetrics.java @@ -189,7 +189,7 @@ public class TestRegionServerMetrics { TEST_UTIL.createTable(tableName, cf); - HTable t = new HTable(conf, tableName); + Table t = new HTable(conf, tableName); Put p = new Put(row); p.add(cf, qualifier, val); @@ -221,7 +221,7 @@ public class TestRegionServerMetrics { TEST_UTIL.createTable(tableName, cf); //Force a hfile. - HTable t = new HTable(conf, tableName); + Table t = new HTable(conf, tableName); Put p = new Put(row); p.add(cf, qualifier, val); t.put(p); @@ -247,7 +247,7 @@ public class TestRegionServerMetrics { byte[] valThree = Bytes.toBytes("ValueThree"); TEST_UTIL.createTable(tableName, cf); - HTable t = new HTable(conf, tableName); + Table t = new HTable(conf, tableName); Put p = new Put(row); p.add(cf, qualifier, valOne); t.put(p); @@ -282,7 +282,7 @@ public class TestRegionServerMetrics { TEST_UTIL.createTable(tableName, cf); - HTable t = new HTable(conf, tableName); + Table t = new HTable(conf, tableName); Put p = new Put(row); p.add(cf, qualifier, val); @@ -314,7 +314,7 @@ public class TestRegionServerMetrics { TEST_UTIL.createTable(tableName, cf); - HTable t = new HTable(conf, tableName); + Table t = new HTable(conf, tableName); Put p = new Put(row); p.add(cf, qualifier, val); diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestSCVFWithMiniCluster.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestSCVFWithMiniCluster.java index c4537232bba..5866352820b 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestSCVFWithMiniCluster.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestSCVFWithMiniCluster.java @@ -35,12 +35,12 @@ import org.apache.hadoop.hbase.TableName; import org.apache.hadoop.hbase.TableNotFoundException; import org.apache.hadoop.hbase.client.Admin; import org.apache.hadoop.hbase.client.Durability; -import org.apache.hadoop.hbase.client.HBaseAdmin; import org.apache.hadoop.hbase.client.HTable; import org.apache.hadoop.hbase.client.Put; import org.apache.hadoop.hbase.client.Result; import org.apache.hadoop.hbase.client.ResultScanner; import org.apache.hadoop.hbase.client.Scan; +import org.apache.hadoop.hbase.client.Table; import org.apache.hadoop.hbase.filter.BinaryComparator; import org.apache.hadoop.hbase.filter.CompareFilter.CompareOp; import org.apache.hadoop.hbase.filter.Filter; @@ -66,7 +66,7 @@ public class TestSCVFWithMiniCluster { private static final byte[] QUALIFIER_FOO = Bytes.toBytes("foo"); private static final byte[] QUALIFIER_BAR = Bytes.toBytes("bar"); - private static HTable htable; + private static Table htable; private static Filter scanFilter; diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestScannerWithBulkload.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestScannerWithBulkload.java index af4b9c560e9..1bf6b761176 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestScannerWithBulkload.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestScannerWithBulkload.java @@ -32,6 +32,7 @@ import org.apache.hadoop.hbase.HTableDescriptor; import org.apache.hadoop.hbase.KeyValue; import org.apache.hadoop.hbase.MediumTests; 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.HTable; import org.apache.hadoop.hbase.client.Put; @@ -56,7 +57,7 @@ public class TestScannerWithBulkload { TEST_UTIL.startMiniCluster(1); } - private static void createTable(HBaseAdmin admin, String tableName) throws IOException { + private static void createTable(Admin admin, String tableName) throws IOException { HTableDescriptor desc = new HTableDescriptor(tableName); HColumnDescriptor hcd = new HColumnDescriptor("col"); hcd.setMaxVersions(3); diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestServerCustomProtocol.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestServerCustomProtocol.java index bfebaa20402..4f711554e31 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestServerCustomProtocol.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestServerCustomProtocol.java @@ -38,6 +38,8 @@ import org.apache.hadoop.hbase.MediumTests; import org.apache.hadoop.hbase.ServerName; import org.apache.hadoop.hbase.client.HTable; import org.apache.hadoop.hbase.client.Put; +import org.apache.hadoop.hbase.client.RegionLocator; +import org.apache.hadoop.hbase.client.Table; import org.apache.hadoop.hbase.client.coprocessor.Batch; import org.apache.hadoop.hbase.coprocessor.CoprocessorException; import org.apache.hadoop.hbase.coprocessor.CoprocessorHost; @@ -181,7 +183,7 @@ public class TestServerCustomProtocol { @Test public void testSingleProxy() throws Throwable { - HTable table = new HTable(util.getConfiguration(), TEST_TABLE); + Table table = new HTable(util.getConfiguration(), TEST_TABLE); Map results = ping(table, null, null); // There are three regions so should get back three results. assertEquals(3, results.size()); @@ -231,7 +233,7 @@ public class TestServerCustomProtocol { table.close(); } - private Map hello(final HTable table, final String send, final String response) + private Map hello(final Table table, final String send, final String response) throws ServiceException, Throwable { Map results = hello(table, send); for (Map.Entry e: results.entrySet()) { @@ -240,12 +242,12 @@ public class TestServerCustomProtocol { return results; } - private Map hello(final HTable table, final String send) + private Map hello(final Table table, final String send) throws ServiceException, Throwable { return hello(table, send, null, null); } - private Map hello(final HTable table, final String send, final byte [] start, + private Map hello(final Table table, final String send, final byte [] start, final byte [] end) throws ServiceException, Throwable { return table.coprocessorService(PingProtos.PingService.class, @@ -264,7 +266,7 @@ public class TestServerCustomProtocol { }); } - private Map compoundOfHelloAndPing(final HTable table, final byte [] start, + private Map compoundOfHelloAndPing(final Table table, final byte [] start, final byte [] end) throws ServiceException, Throwable { return table.coprocessorService(PingProtos.PingService.class, @@ -284,7 +286,7 @@ public class TestServerCustomProtocol { }); } - private Map noop(final HTable table, final byte [] start, + private Map noop(final Table table, final byte [] start, final byte [] end) throws ServiceException, Throwable { return table.coprocessorService(PingProtos.PingService.class, start, end, @@ -391,7 +393,7 @@ public class TestServerCustomProtocol { table.close(); } - private Map ping(final HTable table, final byte [] start, final byte [] end) + private Map ping(final Table table, final byte [] start, final byte [] end) throws ServiceException, Throwable { return table.coprocessorService(PingProtos.PingService.class, start, end, new Batch.Call() { @@ -439,7 +441,7 @@ public class TestServerCustomProtocol { @Test public void testEmptyReturnType() throws Throwable { - HTable table = new HTable(util.getConfiguration(), TEST_TABLE); + Table table = new HTable(util.getConfiguration(), TEST_TABLE); Map results = noop(table, ROW_A, ROW_C); assertEquals("Should have results from three regions", 3, results.size()); // all results should be null @@ -448,12 +450,12 @@ public class TestServerCustomProtocol { } } - private void verifyRegionResults(HTable table, + private void verifyRegionResults(RegionLocator table, Map results, byte[] row) throws Exception { verifyRegionResults(table, results, "pong", row); } - private void verifyRegionResults(HTable table, + private void verifyRegionResults(RegionLocator table, Map results, String expected, byte[] row) throws Exception { for (Map.Entry e: results.entrySet()) { diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestSplitTransactionOnCluster.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestSplitTransactionOnCluster.java index aa628dc5502..edbdd64a05c 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestSplitTransactionOnCluster.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestSplitTransactionOnCluster.java @@ -57,6 +57,8 @@ import org.apache.hadoop.hbase.UnknownRegionException; import org.apache.hadoop.hbase.Waiter; import org.apache.hadoop.hbase.ZooKeeperConnectionException; import org.apache.hadoop.hbase.MetaTableAccessor; +import org.apache.hadoop.hbase.client.Admin; +import org.apache.hadoop.hbase.client.Consistency; import org.apache.hadoop.hbase.client.Delete; import org.apache.hadoop.hbase.client.HBaseAdmin; import org.apache.hadoop.hbase.client.HTable; @@ -67,6 +69,8 @@ import org.apache.hadoop.hbase.client.ResultScanner; import org.apache.hadoop.hbase.client.Scan; import org.apache.hadoop.hbase.coordination.ZKSplitTransactionCoordination; import org.apache.hadoop.hbase.coordination.ZkCloseRegionCoordination; +import org.apache.hadoop.hbase.client.Table; +import org.apache.hadoop.hbase.client.TestReplicasClient.SlowMeCopro; import org.apache.hadoop.hbase.coordination.ZkCoordinatedStateManager; import org.apache.hadoop.hbase.coprocessor.BaseRegionObserver; import org.apache.hadoop.hbase.coprocessor.ObserverContext; @@ -278,7 +282,7 @@ public class TestSplitTransactionOnCluster { TableName.valueOf("testRITStateForRollback"); try { // Create table then get the single region for our new table. - HTable t = createTableAndWait(tableName.getName(), Bytes.toBytes("cf")); + Table t = createTableAndWait(tableName.getName(), Bytes.toBytes("cf")); final List regions = cluster.getRegions(tableName); final HRegionInfo hri = getAndCheckSingleTableRegion(regions); insertData(tableName.getName(), admin, t); @@ -340,7 +344,7 @@ public class TestSplitTransactionOnCluster { int regionServerIndex = cluster.getServerWith(region.getRegionName()); HRegionServer regionServer = cluster.getRegionServer(regionServerIndex); - HTable t = new HTable(conf, tableName); + Table t = new HTable(conf, tableName); // insert data insertData(tableName, admin, t); insertData(tableName, admin, t); @@ -624,7 +628,7 @@ public class TestSplitTransactionOnCluster { HColumnDescriptor hcd = new HColumnDescriptor("col"); htd.addFamily(hcd); admin.createTable(htd); - HTable table = new HTable(conf, userTableName); + Table table = new HTable(conf, userTableName); try { for (int i = 0; i <= 5; i++) { String row = "row" + i; @@ -886,7 +890,7 @@ public class TestSplitTransactionOnCluster { final TableName tableName = TableName.valueOf("testTableExistsIfTheSpecifiedTableRegionIsSplitParent"); // Create table then get the single region for our new table. - HTable t = createTableAndWait(tableName.getName(), Bytes.toBytes("cf")); + Table t = createTableAndWait(tableName.getName(), Bytes.toBytes("cf")); List regions = null; try { regions = cluster.getRegions(tableName); @@ -924,7 +928,7 @@ public class TestSplitTransactionOnCluster { } } - private void insertData(final byte[] tableName, HBaseAdmin admin, HTable t) throws IOException, + private void insertData(final byte[] tableName, HBaseAdmin admin, Table t) throws IOException, InterruptedException { Put p = new Put(Bytes.toBytes("row1")); p.add(Bytes.toBytes("cf"), Bytes.toBytes("q1"), Bytes.toBytes("1")); @@ -1044,8 +1048,8 @@ public class TestSplitTransactionOnCluster { cluster.getServerHoldingRegion(firstTableregions.get(0).getRegionName()); admin.move(secondTableRegions.get(0).getRegionInfo().getEncodedNameAsBytes(), Bytes.toBytes(serverName.getServerName())); - HTable table1 = null; - HTable table2 = null; + Table table1 = null; + Table table2 = null; try { table1 = new HTable(TESTING_UTIL.getConfiguration(), firstTable); table2 = new HTable(TESTING_UTIL.getConfiguration(), firstTable); @@ -1258,7 +1262,7 @@ public class TestSplitTransactionOnCluster { * @throws org.apache.hadoop.hbase.ZooKeeperConnectionException * @throws InterruptedException */ - private int ensureTableRegionNotOnSameServerAsMeta(final HBaseAdmin admin, + private int ensureTableRegionNotOnSameServerAsMeta(final Admin admin, final HRegionInfo hri) throws HBaseIOException, MasterNotRunningException, ZooKeeperConnectionException, InterruptedException { diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestTags.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestTags.java index e612585e319..c628d10ca67 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestTags.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestTags.java @@ -47,6 +47,7 @@ import org.apache.hadoop.hbase.client.Put; import org.apache.hadoop.hbase.client.Result; import org.apache.hadoop.hbase.client.ResultScanner; import org.apache.hadoop.hbase.client.Scan; +import org.apache.hadoop.hbase.client.Table; import org.apache.hadoop.hbase.coprocessor.BaseRegionObserver; import org.apache.hadoop.hbase.coprocessor.CoprocessorHost; import org.apache.hadoop.hbase.coprocessor.ObserverContext; @@ -96,7 +97,7 @@ public class TestTags { @Test public void testTags() throws Exception { - HTable table = null; + Table table = null; try { TableName tableName = TableName.valueOf(TEST_NAME.getMethodName()); byte[] fam = Bytes.toBytes("info"); @@ -174,7 +175,7 @@ public class TestTags { @Test public void testFlushAndCompactionWithoutTags() throws Exception { - HTable table = null; + Table table = null; try { TableName tableName = TableName.valueOf(TEST_NAME.getMethodName()); byte[] fam = Bytes.toBytes("info"); @@ -288,7 +289,7 @@ public class TestTags { byte[] row2 = Bytes.toBytes("rowc"); byte[] rowd = Bytes.toBytes("rowd"); byte[] rowe = Bytes.toBytes("rowe"); - HTable table = null; + Table table = null; for (DataBlockEncoding encoding : DataBlockEncoding.values()) { HTableDescriptor desc = new HTableDescriptor(tableName); HColumnDescriptor colDesc = new HColumnDescriptor(fam); @@ -420,7 +421,7 @@ public class TestTags { desc.addFamily(colDesc); TEST_UTIL.getHBaseAdmin().createTable(desc); - HTable table = null; + Table table = null; try { table = new HTable(TEST_UTIL.getConfiguration(), tableName); Put put = new Put(row1); @@ -541,7 +542,7 @@ public class TestTags { } } - private void result(byte[] fam, byte[] row, byte[] qual, byte[] row2, HTable table, byte[] value, + private void result(byte[] fam, byte[] row, byte[] qual, byte[] row2, Table table, byte[] value, byte[] value2, byte[] row1, byte[] value1) throws IOException { Scan s = new Scan(row); // If filters are used this attribute can be specifically check for in diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/TestHLogFiltering.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/TestHLogFiltering.java index cc8c5317556..6e9f7d5d976 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/TestHLogFiltering.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/TestHLogFiltering.java @@ -30,8 +30,8 @@ import org.apache.hadoop.hbase.HBaseTestingUtility; import org.apache.hadoop.hbase.MediumTests; import org.apache.hadoop.hbase.TableName; import org.apache.hadoop.hbase.client.Delete; -import org.apache.hadoop.hbase.client.HTable; import org.apache.hadoop.hbase.client.Put; +import org.apache.hadoop.hbase.client.Table; import org.apache.hadoop.hbase.master.HMaster; import org.apache.hadoop.hbase.protobuf.RequestConverter; import org.apache.hadoop.hbase.protobuf.generated.AdminProtos.FlushRegionRequest; @@ -72,7 +72,7 @@ public class TestHLogFiltering { } private void fillTable() throws IOException, InterruptedException { - HTable table = TEST_UTIL.createTable(TABLE_NAME, FAMILIES, 3, + Table table = TEST_UTIL.createTable(TABLE_NAME, FAMILIES, 3, Bytes.toBytes("row0"), Bytes.toBytes("row99"), NUM_RS); Random rand = new Random(19387129L); for (int iStoreFile = 0; iStoreFile < 4; ++iStoreFile) { diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/TestLogRollAbort.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/TestLogRollAbort.java index 9fc9f09f6d3..ac71a3d649b 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/TestLogRollAbort.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/TestLogRollAbort.java @@ -17,6 +17,12 @@ */ package org.apache.hadoop.hbase.regionserver.wal; +import java.io.FileNotFoundException; +import java.io.IOException; +import java.util.concurrent.atomic.AtomicLong; + +import org.apache.hadoop.hbase.client.Table; +import org.junit.Assert; import static org.junit.Assert.assertTrue; import org.apache.commons.logging.Log; @@ -29,7 +35,6 @@ import org.apache.hadoop.hbase.MediumTests; import org.apache.hadoop.hbase.MiniHBaseCluster; import org.apache.hadoop.hbase.TableName; import org.apache.hadoop.hbase.client.Admin; -import org.apache.hadoop.hbase.client.HBaseAdmin; import org.apache.hadoop.hbase.client.HTable; import org.apache.hadoop.hbase.client.Put; import org.apache.hadoop.hbase.regionserver.HRegionServer; @@ -112,7 +117,7 @@ public class TestLogRollAbort { desc.addFamily(new HColumnDescriptor(HConstants.CATALOG_FAMILY)); admin.createTable(desc); - HTable table = new HTable(TEST_UTIL.getConfiguration(), tableName); + Table table = new HTable(TEST_UTIL.getConfiguration(), tableName); try { HRegionServer server = TEST_UTIL.getRSForFirstRegionInTable(Bytes.toBytes(tableName)); diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/TestLogRollPeriod.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/TestLogRollPeriod.java index 1c267265d36..06b5f489cb0 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/TestLogRollPeriod.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/TestLogRollPeriod.java @@ -27,18 +27,14 @@ import org.apache.commons.logging.Log; import org.apache.commons.logging.LogFactory; import org.apache.hadoop.fs.Path; import org.apache.hadoop.hbase.HBaseTestingUtility; -import org.apache.hadoop.hbase.HColumnDescriptor; import org.apache.hadoop.hbase.HRegionInfo; import org.apache.hadoop.hbase.HTableDescriptor; import org.apache.hadoop.hbase.MediumTests; -import org.apache.hadoop.hbase.TableName; import org.apache.hadoop.hbase.client.HTable; import org.apache.hadoop.hbase.client.Put; +import org.apache.hadoop.hbase.client.Table; import org.apache.hadoop.hbase.regionserver.HRegionServer; -import org.apache.hadoop.hbase.regionserver.wal.HLog; import org.apache.hadoop.hbase.util.Bytes; -import org.apache.hadoop.hbase.util.FSUtils; -import org.apache.hadoop.hdfs.MiniDFSCluster; import org.junit.AfterClass; import org.junit.BeforeClass; import org.junit.Test; @@ -79,7 +75,7 @@ public class TestLogRollPeriod { TEST_UTIL.createTable(tableName, "cf"); try { - HTable table = new HTable(TEST_UTIL.getConfiguration(), tableName); + Table table = new HTable(TEST_UTIL.getConfiguration(), tableName); try { HRegionServer server = TEST_UTIL.getRSForFirstRegionInTable(Bytes.toBytes(tableName)); HLog log = server.getWAL(); @@ -104,7 +100,7 @@ public class TestLogRollPeriod { try { HRegionServer server = TEST_UTIL.getRSForFirstRegionInTable(Bytes.toBytes(tableName)); HLog log = server.getWAL(); - final HTable table = new HTable(TEST_UTIL.getConfiguration(), tableName); + final Table table = new HTable(TEST_UTIL.getConfiguration(), tableName); Thread writerThread = new Thread("writer") { @Override diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/TestLogRolling.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/TestLogRolling.java index 2a6f4f50b6c..1a4b08a1045 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/TestLogRolling.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/TestLogRolling.java @@ -54,6 +54,7 @@ import org.apache.hadoop.hbase.client.Put; import org.apache.hadoop.hbase.client.Result; import org.apache.hadoop.hbase.client.ResultScanner; import org.apache.hadoop.hbase.client.Scan; +import org.apache.hadoop.hbase.client.Table; import org.apache.hadoop.hbase.fs.HFileSystem; import org.apache.hadoop.hbase.regionserver.HRegion; import org.apache.hadoop.hbase.regionserver.HRegionServer; @@ -175,7 +176,7 @@ public class TestLogRolling { this.server = cluster.getRegionServerThreads().get(0).getRegionServer(); this.log = server.getWAL(); - HTable table = createTestTable(this.tableName); + Table table = createTestTable(this.tableName); server = TEST_UTIL.getRSForFirstRegionInTable(Bytes.toBytes(tableName)); this.log = server.getWAL(); @@ -242,7 +243,7 @@ public class TestLogRolling { return "TestLogRolling"; } - void writeData(HTable table, int rownum) throws IOException { + void writeData(Table table, int rownum) throws IOException { doPut(table, rownum); // sleep to let the log roller run (if it needs to) @@ -253,7 +254,7 @@ public class TestLogRolling { } } - void validateData(HTable table, int rownum) throws IOException { + void validateData(Table table, int rownum) throws IOException { String row = "row" + String.format("%1$04d", rownum); Get get = new Get(Bytes.toBytes(row)); get.addFamily(HConstants.CATALOG_FAMILY); @@ -264,7 +265,7 @@ public class TestLogRolling { LOG.info("Validated row " + row); } - void batchWriteAndWait(HTable table, int start, boolean expect, int timeout) + void batchWriteAndWait(Table table, int start, boolean expect, int timeout) throws IOException { for (int i = 0; i < 10; i++) { Put put = new Put(Bytes.toBytes("row" @@ -335,7 +336,7 @@ public class TestLogRolling { desc.addFamily(new HColumnDescriptor(HConstants.CATALOG_FAMILY)); admin.createTable(desc); - HTable table = new HTable(TEST_UTIL.getConfiguration(), tableName); + Table table = new HTable(TEST_UTIL.getConfiguration(), tableName); assertTrue(table.isAutoFlush()); server = TEST_UTIL.getRSForFirstRegionInTable(Bytes.toBytes(tableName)); @@ -432,7 +433,7 @@ public class TestLogRolling { LOG.info("Replication=" + fs.getDefaultReplication(TEST_UTIL.getDataTestDirOnTestFS())); // When the hbase:meta table can be opened, the region servers are running - HTable t = new HTable(TEST_UTIL.getConfiguration(), TableName.META_TABLE_NAME); + Table t = new HTable(TEST_UTIL.getConfiguration(), TableName.META_TABLE_NAME); try { this.server = cluster.getRegionServer(0); this.log = server.getWAL(); @@ -591,11 +592,11 @@ public class TestLogRolling { */ @Test public void testCompactionRecordDoesntBlockRolling() throws Exception { - HTable table = null; - HTable table2 = null; + Table table = null; + Table table2 = null; // When the hbase:meta table can be opened, the region servers are running - HTable t = new HTable(TEST_UTIL.getConfiguration(), TableName.META_TABLE_NAME); + Table t = new HTable(TEST_UTIL.getConfiguration(), TableName.META_TABLE_NAME); try { String tableName = getName(); table = createTestTable(tableName); @@ -649,13 +650,13 @@ public class TestLogRolling { } } - private void doPut(HTable table, int i) throws IOException { + private void doPut(Table table, int i) throws IOException { Put put = new Put(Bytes.toBytes("row" + String.format("%1$04d", i))); put.add(HConstants.CATALOG_FAMILY, null, value); table.put(put); } - private HTable createTestTable(String tableName) throws IOException { + private Table createTestTable(String tableName) throws IOException { // Create the test table and open it HTableDescriptor desc = new HTableDescriptor(TableName.valueOf(tableName)); desc.addFamily(new HColumnDescriptor(HConstants.CATALOG_FAMILY)); diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/TestWALReplay.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/TestWALReplay.java index 5b1393105cb..7e129e23d06 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/TestWALReplay.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/TestWALReplay.java @@ -58,6 +58,7 @@ import org.apache.hadoop.hbase.client.Put; import org.apache.hadoop.hbase.client.Result; import org.apache.hadoop.hbase.client.ResultScanner; import org.apache.hadoop.hbase.client.Scan; +import org.apache.hadoop.hbase.client.Table; import org.apache.hadoop.hbase.master.HMaster; import org.apache.hadoop.hbase.monitoring.MonitoredTask; import org.apache.hadoop.hbase.protobuf.generated.ZooKeeperProtos.SplitLogTask.RecoveryMode; @@ -166,7 +167,7 @@ public class TestWALReplay { byte[] value = Bytes.toBytes("testV"); byte[][] familys = { family1, family2 }; TEST_UTIL.createTable(tableName, familys); - HTable htable = new HTable(TEST_UTIL.getConfiguration(), tableName); + Table htable = new HTable(TEST_UTIL.getConfiguration(), tableName); Put put = new Put(Bytes.toBytes("r1")); put.add(family1, qualifier, value); htable.put(put); diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestMasterReplication.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestMasterReplication.java index 72a1513f191..efff6dd5443 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestMasterReplication.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestMasterReplication.java @@ -37,6 +37,7 @@ import org.apache.hadoop.hbase.HTableDescriptor; import org.apache.hadoop.hbase.KeyValue; import org.apache.hadoop.hbase.LargeTests; import org.apache.hadoop.hbase.TableName; +import org.apache.hadoop.hbase.client.Admin; import org.apache.hadoop.hbase.client.Delete; import org.apache.hadoop.hbase.client.Durability; import org.apache.hadoop.hbase.client.Get; @@ -44,6 +45,7 @@ import org.apache.hadoop.hbase.client.HBaseAdmin; import org.apache.hadoop.hbase.client.HTable; import org.apache.hadoop.hbase.client.Put; import org.apache.hadoop.hbase.client.Result; +import org.apache.hadoop.hbase.client.Table; import org.apache.hadoop.hbase.client.replication.ReplicationAdmin; import org.apache.hadoop.hbase.coprocessor.BaseRegionObserver; import org.apache.hadoop.hbase.coprocessor.CoprocessorHost; @@ -123,7 +125,7 @@ public class TestMasterReplication { public void testCyclicReplication1() throws Exception { LOG.info("testSimplePutDelete"); int numClusters = 2; - HTable[] htables = null; + Table[] htables = null; try { startMiniClusters(numClusters); createTableOnClusters(table); @@ -163,7 +165,7 @@ public class TestMasterReplication { public void testCyclicReplication2() throws Exception { LOG.info("testCyclicReplication1"); int numClusters = 3; - HTable[] htables = null; + Table[] htables = null; try { startMiniClusters(numClusters); createTableOnClusters(table); @@ -214,7 +216,7 @@ public class TestMasterReplication { public void testCyclicReplication3() throws Exception { LOG.info("testCyclicReplication2"); int numClusters = 3; - HTable[] htables = null; + Table[] htables = null; try { startMiniClusters(numClusters); createTableOnClusters(table); @@ -285,7 +287,7 @@ public class TestMasterReplication { private void createTableOnClusters(HTableDescriptor table) throws Exception { int numClusters = configurations.length; for (int i = 0; i < numClusters; i++) { - HBaseAdmin hbaseAdmin = null; + Admin hbaseAdmin = null; try { hbaseAdmin = new HBaseAdmin(configurations[i]); hbaseAdmin.createTable(table); @@ -343,18 +345,18 @@ public class TestMasterReplication { } @SuppressWarnings("resource") - private HTable[] getHTablesOnClusters(byte[] tableName) throws Exception { + private Table[] getHTablesOnClusters(byte[] tableName) throws Exception { int numClusters = utilities.length; - HTable[] htables = new HTable[numClusters]; + Table[] htables = new Table[numClusters]; for (int i = 0; i < numClusters; i++) { - HTable htable = new HTable(configurations[i], tableName); + Table htable = new HTable(configurations[i], tableName); htable.setWriteBufferSize(1024); htables[i] = htable; } return htables; } - private void validateCounts(HTable[] htables, byte[] type, + private void validateCounts(Table[] htables, byte[] type, int[] expectedCounts) throws IOException { for (int i = 0; i < htables.length; i++) { assertEquals(Bytes.toString(type) + " were replicated back ", @@ -362,21 +364,21 @@ public class TestMasterReplication { } } - private int getCount(HTable t, byte[] type) throws IOException { + private int getCount(Table t, byte[] type) throws IOException { Get test = new Get(row); test.setAttribute("count", new byte[] {}); Result res = t.get(test); return Bytes.toInt(res.getValue(count, type)); } - private void deleteAndWait(byte[] row, HTable source, HTable target) + private void deleteAndWait(byte[] row, Table source, Table target) throws Exception { Delete del = new Delete(row); source.delete(del); wait(row, target, true); } - private void putAndWait(byte[] row, byte[] fam, HTable source, HTable target) + private void putAndWait(byte[] row, byte[] fam, Table source, Table target) throws Exception { Put put = new Put(row); put.add(fam, row, row); @@ -384,7 +386,7 @@ public class TestMasterReplication { wait(row, target, false); } - private void wait(byte[] row, HTable target, boolean isDeleted) + private void wait(byte[] row, Table target, boolean isDeleted) throws Exception { Get get = new Get(row); for (int i = 0; i < NB_RETRIES; i++) { diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestMultiSlaveReplication.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestMultiSlaveReplication.java index f3daa97d50b..5abfeea9d71 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestMultiSlaveReplication.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestMultiSlaveReplication.java @@ -35,6 +35,7 @@ import org.apache.hadoop.hbase.client.HBaseAdmin; import org.apache.hadoop.hbase.client.HTable; import org.apache.hadoop.hbase.client.Put; import org.apache.hadoop.hbase.client.Result; +import org.apache.hadoop.hbase.client.Table; import org.apache.hadoop.hbase.client.replication.ReplicationAdmin; import org.apache.hadoop.hbase.coprocessor.CoprocessorHost; import org.apache.hadoop.hbase.util.Bytes; @@ -124,11 +125,11 @@ public class TestMultiSlaveReplication { new HBaseAdmin(conf1).createTable(table); new HBaseAdmin(conf2).createTable(table); new HBaseAdmin(conf3).createTable(table); - HTable htable1 = new HTable(conf1, tableName); + Table htable1 = new HTable(conf1, tableName); htable1.setWriteBufferSize(1024); - HTable htable2 = new HTable(conf2, tableName); + Table htable2 = new HTable(conf2, tableName); htable2.setWriteBufferSize(1024); - HTable htable3 = new HTable(conf3, tableName); + Table htable3 = new HTable(conf3, tableName); htable3.setWriteBufferSize(1024); admin1.addPeer("1", utility2.getClusterKey()); @@ -187,7 +188,7 @@ public class TestMultiSlaveReplication { utility1.shutdownMiniCluster(); } - private void checkWithWait(byte[] row, int count, HTable table) throws Exception { + private void checkWithWait(byte[] row, int count, Table table) throws Exception { Get get = new Get(row); for (int i = 0; i < NB_RETRIES; i++) { if (i == NB_RETRIES - 1) { @@ -209,15 +210,15 @@ public class TestMultiSlaveReplication { } } - private void checkRow(byte[] row, int count, HTable... tables) throws IOException { + private void checkRow(byte[] row, int count, Table... tables) throws IOException { Get get = new Get(row); - for (HTable table : tables) { + for (Table table : tables) { Result res = table.get(get); assertEquals(count, res.size()); } } - private void deleteAndWait(byte[] row, HTable source, HTable... targets) + private void deleteAndWait(byte[] row, Table source, Table... targets) throws Exception { Delete del = new Delete(row); source.delete(del); @@ -228,7 +229,7 @@ public class TestMultiSlaveReplication { fail("Waited too much time for del replication"); } boolean removedFromAll = true; - for (HTable target : targets) { + for (Table target : targets) { Result res = target.get(get); if (res.size() >= 1) { LOG.info("Row not deleted"); @@ -244,7 +245,7 @@ public class TestMultiSlaveReplication { } } - private void putAndWait(byte[] row, byte[] fam, HTable source, HTable... targets) + private void putAndWait(byte[] row, byte[] fam, Table source, Table... targets) throws Exception { Put put = new Put(row); put.add(fam, row, row); @@ -256,7 +257,7 @@ public class TestMultiSlaveReplication { fail("Waited too much time for put replication"); } boolean replicatedToAll = true; - for (HTable target : targets) { + for (Table target : targets) { Result res = target.get(get); if (res.size() == 0) { LOG.info("Row not available"); diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestPerTableCFReplication.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestPerTableCFReplication.java index ff77a94ac2d..bc3c38fd53c 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestPerTableCFReplication.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestPerTableCFReplication.java @@ -39,6 +39,7 @@ import org.apache.hadoop.hbase.client.HBaseAdmin; import org.apache.hadoop.hbase.client.HTable; import org.apache.hadoop.hbase.client.Put; import org.apache.hadoop.hbase.client.Result; +import org.apache.hadoop.hbase.client.Table; import org.apache.hadoop.hbase.client.replication.ReplicationAdmin; import org.apache.hadoop.hbase.coprocessor.CoprocessorHost; import org.apache.hadoop.hbase.util.Bytes; @@ -272,17 +273,17 @@ public class TestPerTableCFReplication { new HBaseAdmin(conf3).createTable(tabB); new HBaseAdmin(conf3).createTable(tabC); - HTable htab1A = new HTable(conf1, tabAName); - HTable htab2A = new HTable(conf2, tabAName); - HTable htab3A = new HTable(conf3, tabAName); + Table htab1A = new HTable(conf1, tabAName); + Table htab2A = new HTable(conf2, tabAName); + Table htab3A = new HTable(conf3, tabAName); - HTable htab1B = new HTable(conf1, tabBName); - HTable htab2B = new HTable(conf2, tabBName); - HTable htab3B = new HTable(conf3, tabBName); + Table htab1B = new HTable(conf1, tabBName); + Table htab2B = new HTable(conf2, tabBName); + Table htab3B = new HTable(conf3, tabBName); - HTable htab1C = new HTable(conf1, tabCName); - HTable htab2C = new HTable(conf2, tabCName); - HTable htab3C = new HTable(conf3, tabCName); + Table htab1C = new HTable(conf1, tabCName); + Table htab2C = new HTable(conf2, tabCName); + Table htab3C = new HTable(conf3, tabCName); // A. add cluster2/cluster3 as peers to cluster1 admin1.addPeer("2", utility2.getClusterKey(), "TC;TB:f1,f3"); @@ -371,17 +372,17 @@ public class TestPerTableCFReplication { deleteAndWaitWithFamily(row2, f3Name, htab1C, htab2C, htab3C); } - private void ensureRowNotReplicated(byte[] row, byte[] fam, HTable... tables) throws IOException { + private void ensureRowNotReplicated(byte[] row, byte[] fam, Table... tables) throws IOException { Get get = new Get(row); get.addFamily(fam); - for (HTable table : tables) { + for (Table table : tables) { Result res = table.get(get); assertEquals(0, res.size()); } } private void deleteAndWaitWithFamily(byte[] row, byte[] fam, - HTable source, HTable... targets) + Table source, Table... targets) throws Exception { Delete del = new Delete(row); del.deleteFamily(fam); @@ -394,7 +395,7 @@ public class TestPerTableCFReplication { fail("Waited too much time for del replication"); } boolean removedFromAll = true; - for (HTable target : targets) { + for (Table target : targets) { Result res = target.get(get); if (res.size() >= 1) { LOG.info("Row not deleted"); @@ -411,7 +412,7 @@ public class TestPerTableCFReplication { } private void putAndWaitWithFamily(byte[] row, byte[] fam, - HTable source, HTable... targets) + Table source, Table... targets) throws Exception { Put put = new Put(row); put.add(fam, row, val); @@ -424,7 +425,7 @@ public class TestPerTableCFReplication { fail("Waited too much time for put replication"); } boolean replicatedToAll = true; - for (HTable target : targets) { + for (Table target : targets) { Result res = target.get(get); if (res.size() == 0) { LOG.info("Row not available"); diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestReplicationBase.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestReplicationBase.java index 080d858b7c9..252d3a49539 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestReplicationBase.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestReplicationBase.java @@ -28,8 +28,10 @@ import org.apache.hadoop.hbase.HColumnDescriptor; import org.apache.hadoop.hbase.HConstants; import org.apache.hadoop.hbase.HTableDescriptor; import org.apache.hadoop.hbase.TableName; +import org.apache.hadoop.hbase.client.Admin; import org.apache.hadoop.hbase.client.HBaseAdmin; import org.apache.hadoop.hbase.client.HTable; +import org.apache.hadoop.hbase.client.Table; import org.apache.hadoop.hbase.client.replication.ReplicationAdmin; import org.apache.hadoop.hbase.replication.regionserver.ReplicationSource; import org.apache.hadoop.hbase.util.Bytes; @@ -63,7 +65,7 @@ public class TestReplicationBase { protected static ReplicationAdmin admin; protected static HTable htable1; - protected static HTable htable2; + protected static Table htable2; protected static HBaseTestingUtility utility1; protected static HBaseTestingUtility utility2; @@ -134,8 +136,8 @@ public class TestReplicationBase { table.addFamily(fam); fam = new HColumnDescriptor(noRepfamName); table.addFamily(fam); - HBaseAdmin admin1 = new HBaseAdmin(conf1); - HBaseAdmin admin2 = new HBaseAdmin(conf2); + Admin admin1 = new HBaseAdmin(conf1); + Admin admin2 = new HBaseAdmin(conf2); admin1.createTable(table, HBaseTestingUtility.KEYS_FOR_HBA_CREATE_TABLE); admin2.createTable(table, HBaseTestingUtility.KEYS_FOR_HBA_CREATE_TABLE); htable1 = new HTable(conf1, tableName); diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestReplicationSyncUpTool.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestReplicationSyncUpTool.java index e8620d47543..aded6365a28 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestReplicationSyncUpTool.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestReplicationSyncUpTool.java @@ -25,10 +25,12 @@ import java.util.List; import org.apache.commons.logging.Log; import org.apache.commons.logging.LogFactory; import org.apache.hadoop.hbase.*; +import org.apache.hadoop.hbase.client.Admin; import org.apache.hadoop.hbase.client.Delete; import org.apache.hadoop.hbase.client.HBaseAdmin; import org.apache.hadoop.hbase.client.HTable; import org.apache.hadoop.hbase.client.Put; +import org.apache.hadoop.hbase.client.Table; import org.apache.hadoop.hbase.client.replication.ReplicationAdmin; import org.apache.hadoop.hbase.util.Bytes; import org.apache.hadoop.hbase.replication.regionserver.ReplicationSyncUp; @@ -52,7 +54,7 @@ public class TestReplicationSyncUpTool extends TestReplicationBase { private HTableDescriptor t1_syncupSource, t1_syncupTarget; private HTableDescriptor t2_syncupSource, t2_syncupTarget; - private HTable ht1Source, ht2Source, ht1TargetAtPeer1, ht2TargetAtPeer1; + private Table ht1Source, ht2Source, ht1TargetAtPeer1, ht2TargetAtPeer1; @Before public void setUp() throws Exception { @@ -175,7 +177,7 @@ public class TestReplicationSyncUpTool extends TestReplicationBase { ReplicationAdmin admin1 = new ReplicationAdmin(conf1); ReplicationAdmin admin2 = new ReplicationAdmin(conf2); - HBaseAdmin ha = new HBaseAdmin(conf1); + Admin ha = new HBaseAdmin(conf1); ha.createTable(t1_syncupSource); ha.createTable(t2_syncupSource); ha.close(); diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestReplicationWithTags.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestReplicationWithTags.java index 491a9db0b2d..9a49e9c7155 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestReplicationWithTags.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestReplicationWithTags.java @@ -39,12 +39,14 @@ import org.apache.hadoop.hbase.KeyValueUtil; import org.apache.hadoop.hbase.LargeTests; import org.apache.hadoop.hbase.TableName; import org.apache.hadoop.hbase.Tag; +import org.apache.hadoop.hbase.client.Admin; import org.apache.hadoop.hbase.client.Durability; import org.apache.hadoop.hbase.client.Get; import org.apache.hadoop.hbase.client.HBaseAdmin; import org.apache.hadoop.hbase.client.HTable; import org.apache.hadoop.hbase.client.Put; import org.apache.hadoop.hbase.client.Result; +import org.apache.hadoop.hbase.client.Table; import org.apache.hadoop.hbase.client.replication.ReplicationAdmin; import org.apache.hadoop.hbase.codec.KeyValueCodecWithTags; import org.apache.hadoop.hbase.coprocessor.BaseRegionObserver; @@ -70,8 +72,8 @@ public class TestReplicationWithTags { private static ReplicationAdmin replicationAdmin; - private static HTable htable1; - private static HTable htable2; + private static Table htable1; + private static Table htable2; private static HBaseTestingUtility utility1; private static HBaseTestingUtility utility2; @@ -134,7 +136,7 @@ public class TestReplicationWithTags { fam.setMaxVersions(3); fam.setScope(HConstants.REPLICATION_SCOPE_GLOBAL); table.addFamily(fam); - HBaseAdmin admin = null; + Admin admin = null; try { admin = new HBaseAdmin(conf1); admin.createTable(table, HBaseTestingUtility.KEYS_FOR_HBA_CREATE_TABLE); diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/regionserver/TestReplicationSink.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/regionserver/TestReplicationSink.java index 3a138e48840..8a08f0c6f49 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/regionserver/TestReplicationSink.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/regionserver/TestReplicationSink.java @@ -25,6 +25,7 @@ import java.util.ArrayList; import java.util.List; import java.util.concurrent.atomic.AtomicBoolean; +import org.apache.hadoop.hbase.client.Table; import org.apache.hadoop.hbase.util.ByteStringer; import org.apache.commons.logging.Log; import org.apache.commons.logging.LogFactory; @@ -37,7 +38,6 @@ import org.apache.hadoop.hbase.KeyValue; import org.apache.hadoop.hbase.MediumTests; import org.apache.hadoop.hbase.Stoppable; import org.apache.hadoop.hbase.client.Get; -import org.apache.hadoop.hbase.client.HTable; import org.apache.hadoop.hbase.client.Result; import org.apache.hadoop.hbase.client.ResultScanner; import org.apache.hadoop.hbase.client.Scan; @@ -69,7 +69,7 @@ public class TestReplicationSink { private static final byte[] FAM_NAME1 = Bytes.toBytes("info1"); private static final byte[] FAM_NAME2 = Bytes.toBytes("info2"); - private static HTable table1; + private static Table table1; private static Stoppable STOPPABLE = new Stoppable() { final AtomicBoolean stop = new AtomicBoolean(false); @@ -86,7 +86,7 @@ public class TestReplicationSink { }; - private static HTable table2; + private static Table table2; /** * @throws java.lang.Exception diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/rest/TestGzipFilter.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/rest/TestGzipFilter.java index 8f8e62ef6b8..d3d6171c071 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/rest/TestGzipFilter.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/rest/TestGzipFilter.java @@ -36,13 +36,12 @@ import org.apache.hadoop.hbase.MediumTests; import org.apache.hadoop.hbase.TableName; import org.apache.hadoop.hbase.client.Admin; import org.apache.hadoop.hbase.client.Get; -import org.apache.hadoop.hbase.client.HBaseAdmin; import org.apache.hadoop.hbase.client.HTable; import org.apache.hadoop.hbase.client.Result; +import org.apache.hadoop.hbase.client.Table; import org.apache.hadoop.hbase.rest.client.Client; import org.apache.hadoop.hbase.rest.client.Cluster; import org.apache.hadoop.hbase.rest.client.Response; -import org.apache.hadoop.hbase.rest.filter.GzipFilter; import org.apache.hadoop.hbase.util.Bytes; import org.junit.AfterClass; import org.junit.BeforeClass; @@ -102,7 +101,7 @@ public class TestGzipFilter { Response response = client.put(path, headers, value_1_gzip); assertEquals(response.getCode(), 200); - HTable table = new HTable(TEST_UTIL.getConfiguration(), TABLE); + Table table = new HTable(TEST_UTIL.getConfiguration(), TABLE); Get get = new Get(Bytes.toBytes(ROW_1)); get.addColumn(Bytes.toBytes(CFA), Bytes.toBytes("1")); Result result = table.get(get); diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/rest/TestScannerResource.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/rest/TestScannerResource.java index a13280e4f5d..e3ed91617cc 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/rest/TestScannerResource.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/rest/TestScannerResource.java @@ -34,10 +34,10 @@ import org.apache.commons.httpclient.Header; import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.hbase.*; import org.apache.hadoop.hbase.client.Admin; -import org.apache.hadoop.hbase.client.HBaseAdmin; import org.apache.hadoop.hbase.client.HTable; import org.apache.hadoop.hbase.client.Put; import org.apache.hadoop.hbase.client.Durability; +import org.apache.hadoop.hbase.client.Table; import org.apache.hadoop.hbase.rest.client.Client; import org.apache.hadoop.hbase.rest.client.Cluster; import org.apache.hadoop.hbase.rest.client.Response; @@ -78,7 +78,7 @@ public class TestScannerResource { throws IOException { Random rng = new Random(); int count = 0; - HTable table = new HTable(conf, tableName); + Table table = new HTable(conf, tableName); byte[] k = new byte[3]; byte [][] famAndQf = KeyValue.parseColumn(Bytes.toBytes(column)); for (byte b1 = 'a'; b1 < 'z'; b1++) { diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/rest/TestScannersWithFilters.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/rest/TestScannersWithFilters.java index 46343351345..e60208b6e1a 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/rest/TestScannersWithFilters.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/rest/TestScannersWithFilters.java @@ -35,11 +35,11 @@ import org.apache.commons.logging.LogFactory; import org.apache.hadoop.hbase.*; import org.apache.hadoop.hbase.client.Admin; import org.apache.hadoop.hbase.client.Delete; -import org.apache.hadoop.hbase.client.HBaseAdmin; import org.apache.hadoop.hbase.client.HTable; import org.apache.hadoop.hbase.client.Put; import org.apache.hadoop.hbase.client.Scan; import org.apache.hadoop.hbase.client.Durability; +import org.apache.hadoop.hbase.client.Table; import org.apache.hadoop.hbase.filter.BinaryComparator; import org.apache.hadoop.hbase.filter.Filter; import org.apache.hadoop.hbase.filter.FilterList; @@ -135,7 +135,7 @@ public class TestScannersWithFilters { htd.addFamily(new HColumnDescriptor(FAMILIES[0])); htd.addFamily(new HColumnDescriptor(FAMILIES[1])); admin.createTable(htd); - HTable table = new HTable(TEST_UTIL.getConfiguration(), TABLE); + Table table = new HTable(TEST_UTIL.getConfiguration(), TABLE); // Insert first half for(byte [] ROW : ROWS_ONE) { Put p = new Put(ROW); diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/rest/TestScannersWithLabels.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/rest/TestScannersWithLabels.java index edf31b8db6e..e8ec7e3b142 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/rest/TestScannersWithLabels.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/rest/TestScannersWithLabels.java @@ -40,9 +40,9 @@ import org.apache.hadoop.hbase.MediumTests; import org.apache.hadoop.hbase.TableName; import org.apache.hadoop.hbase.client.Admin; import org.apache.hadoop.hbase.client.Durability; -import org.apache.hadoop.hbase.client.HBaseAdmin; import org.apache.hadoop.hbase.client.HTable; import org.apache.hadoop.hbase.client.Put; +import org.apache.hadoop.hbase.client.Table; import org.apache.hadoop.hbase.protobuf.generated.VisibilityLabelsProtos.VisibilityLabelsResponse; import org.apache.hadoop.hbase.rest.client.Client; import org.apache.hadoop.hbase.rest.client.Cluster; @@ -90,7 +90,7 @@ public class TestScannersWithLabels { private static int insertData(TableName tableName, String column, double prob) throws IOException { Random rng = new Random(); int count = 0; - HTable table = new HTable(TEST_UTIL.getConfiguration(), tableName); + Table table = new HTable(TEST_UTIL.getConfiguration(), tableName); byte[] k = new byte[3]; byte[][] famAndQf = KeyValue.parseColumn(Bytes.toBytes(column)); diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/rest/client/TestRemoteTable.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/rest/client/TestRemoteTable.java index 120604eead5..cade2dca800 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/rest/client/TestRemoteTable.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/rest/client/TestRemoteTable.java @@ -42,12 +42,12 @@ import org.apache.hadoop.hbase.TableName; import org.apache.hadoop.hbase.client.Admin; import org.apache.hadoop.hbase.client.Delete; import org.apache.hadoop.hbase.client.Get; -import org.apache.hadoop.hbase.client.HBaseAdmin; import org.apache.hadoop.hbase.client.HTable; import org.apache.hadoop.hbase.client.Put; import org.apache.hadoop.hbase.client.Result; import org.apache.hadoop.hbase.client.ResultScanner; import org.apache.hadoop.hbase.client.Scan; +import org.apache.hadoop.hbase.client.Table; import org.apache.hadoop.hbase.rest.HBaseRESTTestingUtility; import org.apache.hadoop.hbase.util.Bytes; import org.junit.After; @@ -99,7 +99,7 @@ public class TestRemoteTable { htd.addFamily(new HColumnDescriptor(COLUMN_2).setMaxVersions(3)); htd.addFamily(new HColumnDescriptor(COLUMN_3).setMaxVersions(3)); admin.createTable(htd); - HTable table = null; + Table table = null; try { table = new HTable(TEST_UTIL.getConfiguration(), TABLE); Put put = new Put(ROW_1); @@ -133,7 +133,7 @@ public class TestRemoteTable { @Test public void testGetTableDescriptor() throws IOException { - HTable table = null; + Table table = null; try { table = new HTable(TEST_UTIL.getConfiguration(), TABLE); HTableDescriptor local = table.getTableDescriptor(); diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/security/access/SecureTestUtil.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/security/access/SecureTestUtil.java index ca27807d5dd..3e834bdaf6b 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/security/access/SecureTestUtil.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/security/access/SecureTestUtil.java @@ -40,6 +40,7 @@ import org.apache.hadoop.hbase.TableName; import org.apache.hadoop.hbase.Waiter.Predicate; import org.apache.hadoop.hbase.client.HTable; 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.io.hfile.HFile; import org.apache.hadoop.hbase.protobuf.ProtobufUtil; @@ -117,7 +118,7 @@ public class SecureTestUtil { for (Permission p : perms) { request.addPermission(ProtobufUtil.toPermission(p)); } - HTable acl = new HTable(conf, table); + Table acl = new HTable(conf, table); try { AccessControlService.BlockingInterface protocol = AccessControlService.newBlockingStub(acl.coprocessorService(new byte[0])); @@ -327,7 +328,7 @@ public class SecureTestUtil { SecureTestUtil.updateACLs(util, new Callable() { @Override public Void call() throws Exception { - HTable acl = new HTable(util.getConfiguration(), AccessControlLists.ACL_TABLE_NAME); + Table acl = new HTable(util.getConfiguration(), AccessControlLists.ACL_TABLE_NAME); try { BlockingRpcChannel service = acl.coprocessorService(HConstants.EMPTY_START_ROW); AccessControlService.BlockingInterface protocol = @@ -351,7 +352,7 @@ public class SecureTestUtil { SecureTestUtil.updateACLs(util, new Callable() { @Override public Void call() throws Exception { - HTable acl = new HTable(util.getConfiguration(), AccessControlLists.ACL_TABLE_NAME); + Table acl = new HTable(util.getConfiguration(), AccessControlLists.ACL_TABLE_NAME); try { BlockingRpcChannel service = acl.coprocessorService(HConstants.EMPTY_START_ROW); AccessControlService.BlockingInterface protocol = @@ -375,7 +376,7 @@ public class SecureTestUtil { SecureTestUtil.updateACLs(util, new Callable() { @Override public Void call() throws Exception { - HTable acl = new HTable(util.getConfiguration(), AccessControlLists.ACL_TABLE_NAME); + Table acl = new HTable(util.getConfiguration(), AccessControlLists.ACL_TABLE_NAME); try { BlockingRpcChannel service = acl.coprocessorService(HConstants.EMPTY_START_ROW); AccessControlService.BlockingInterface protocol = @@ -399,7 +400,7 @@ public class SecureTestUtil { SecureTestUtil.updateACLs(util, new Callable() { @Override public Void call() throws Exception { - HTable acl = new HTable(util.getConfiguration(), AccessControlLists.ACL_TABLE_NAME); + Table acl = new HTable(util.getConfiguration(), AccessControlLists.ACL_TABLE_NAME); try { BlockingRpcChannel service = acl.coprocessorService(HConstants.EMPTY_START_ROW); AccessControlService.BlockingInterface protocol = @@ -424,7 +425,7 @@ public class SecureTestUtil { SecureTestUtil.updateACLs(util, new Callable() { @Override public Void call() throws Exception { - HTable acl = new HTable(util.getConfiguration(), AccessControlLists.ACL_TABLE_NAME); + Table acl = new HTable(util.getConfiguration(), AccessControlLists.ACL_TABLE_NAME); try { BlockingRpcChannel service = acl.coprocessorService(HConstants.EMPTY_START_ROW); AccessControlService.BlockingInterface protocol = @@ -449,7 +450,7 @@ public class SecureTestUtil { SecureTestUtil.updateACLs(util, new Callable() { @Override public Void call() throws Exception { - HTable acl = new HTable(util.getConfiguration(), AccessControlLists.ACL_TABLE_NAME); + Table acl = new HTable(util.getConfiguration(), AccessControlLists.ACL_TABLE_NAME); try { BlockingRpcChannel service = acl.coprocessorService(HConstants.EMPTY_START_ROW); AccessControlService.BlockingInterface protocol = diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/security/access/TestAccessControlFilter.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/security/access/TestAccessControlFilter.java index 427d62dc91f..df358cdac51 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/security/access/TestAccessControlFilter.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/security/access/TestAccessControlFilter.java @@ -36,6 +36,7 @@ import org.apache.hadoop.hbase.client.Put; import org.apache.hadoop.hbase.client.Result; import org.apache.hadoop.hbase.client.ResultScanner; import org.apache.hadoop.hbase.client.Scan; +import org.apache.hadoop.hbase.client.Table; import org.apache.hadoop.hbase.security.User; import org.apache.hadoop.hbase.util.Bytes; import org.junit.AfterClass; @@ -90,7 +91,7 @@ public class TestAccessControlFilter extends SecureTestUtil { @Test public void testQualifierAccess() throws Exception { - final HTable table = TEST_UTIL.createTable(TABLE, FAMILY); + final Table table = TEST_UTIL.createTable(TABLE, FAMILY); try { doQualifierAccess(table); } finally { @@ -98,7 +99,7 @@ public class TestAccessControlFilter extends SecureTestUtil { } } - private void doQualifierAccess(final HTable table) throws Exception { + private void doQualifierAccess(final Table table) throws Exception { // set permissions SecureTestUtil.grantOnTable(TEST_UTIL, READER.getShortName(), TABLE, null, null, Permission.Action.READ); @@ -121,7 +122,7 @@ public class TestAccessControlFilter extends SecureTestUtil { Configuration conf = new Configuration(TEST_UTIL.getConfiguration()); // force a new RS connection conf.set("testkey", UUID.randomUUID().toString()); - HTable t = new HTable(conf, TABLE); + Table t = new HTable(conf, TABLE); try { ResultScanner rs = t.getScanner(new Scan()); int rowcnt = 0; @@ -147,7 +148,7 @@ public class TestAccessControlFilter extends SecureTestUtil { Configuration conf = new Configuration(TEST_UTIL.getConfiguration()); // force a new RS connection conf.set("testkey", UUID.randomUUID().toString()); - HTable t = new HTable(conf, TABLE); + Table t = new HTable(conf, TABLE); try { ResultScanner rs = t.getScanner(new Scan()); int rowcnt = 0; @@ -172,7 +173,7 @@ public class TestAccessControlFilter extends SecureTestUtil { Configuration conf = new Configuration(TEST_UTIL.getConfiguration()); // force a new RS connection conf.set("testkey", UUID.randomUUID().toString()); - HTable t = new HTable(conf, TABLE); + Table t = new HTable(conf, TABLE); try { ResultScanner rs = t.getScanner(new Scan()); int rowcnt = 0; diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/security/access/TestAccessController.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/security/access/TestAccessController.java index 6458a558f27..207576227e2 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/security/access/TestAccessController.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/security/access/TestAccessController.java @@ -53,7 +53,6 @@ import org.apache.hadoop.hbase.client.Admin; import org.apache.hadoop.hbase.client.Append; import org.apache.hadoop.hbase.client.Delete; import org.apache.hadoop.hbase.client.Get; -import org.apache.hadoop.hbase.client.HBaseAdmin; import org.apache.hadoop.hbase.client.HConnection; import org.apache.hadoop.hbase.client.HConnectionManager; import org.apache.hadoop.hbase.client.HTable; @@ -62,6 +61,7 @@ import org.apache.hadoop.hbase.client.Put; import org.apache.hadoop.hbase.client.Result; import org.apache.hadoop.hbase.client.ResultScanner; import org.apache.hadoop.hbase.client.Scan; +import org.apache.hadoop.hbase.client.Table; import org.apache.hadoop.hbase.coprocessor.CoprocessorService; import org.apache.hadoop.hbase.coprocessor.MasterCoprocessorEnvironment; import org.apache.hadoop.hbase.coprocessor.ObserverContext; @@ -98,7 +98,6 @@ import org.apache.hadoop.hbase.security.access.Permission.Action; import org.apache.hadoop.hbase.util.Bytes; import org.apache.hadoop.hbase.util.JVMClusterUtil; import org.apache.hadoop.hbase.util.TestTableName; -import org.apache.hadoop.hbase.security.access.AccessControlClient; import org.apache.log4j.Level; import org.apache.log4j.Logger; import org.junit.After; @@ -691,7 +690,7 @@ public class TestAccessController extends SecureTestUtil { public Object run() throws Exception { Get g = new Get(TEST_ROW); g.addFamily(TEST_FAMILY); - HTable t = new HTable(conf, TEST_TABLE.getTableName()); + Table t = new HTable(conf, TEST_TABLE.getTableName()); try { t.get(g); } finally { @@ -709,7 +708,7 @@ public class TestAccessController extends SecureTestUtil { Scan s = new Scan(); s.addFamily(TEST_FAMILY); - HTable table = new HTable(conf, TEST_TABLE.getTableName()); + Table table = new HTable(conf, TEST_TABLE.getTableName()); try { ResultScanner scanner = table.getScanner(s); try { @@ -738,7 +737,7 @@ public class TestAccessController extends SecureTestUtil { public Object run() throws Exception { Put p = new Put(TEST_ROW); p.add(TEST_FAMILY, TEST_QUALIFIER, Bytes.toBytes(1)); - HTable t = new HTable(conf, TEST_TABLE.getTableName()); + Table t = new HTable(conf, TEST_TABLE.getTableName()); try { t.put(p); } finally { @@ -755,7 +754,7 @@ public class TestAccessController extends SecureTestUtil { public Object run() throws Exception { Delete d = new Delete(TEST_ROW); d.deleteFamily(TEST_FAMILY); - HTable t = new HTable(conf, TEST_TABLE.getTableName()); + Table t = new HTable(conf, TEST_TABLE.getTableName()); try { t.delete(d); } finally { @@ -772,7 +771,7 @@ public class TestAccessController extends SecureTestUtil { public Object run() throws Exception { Increment inc = new Increment(TEST_ROW); inc.addColumn(TEST_FAMILY, TEST_QUALIFIER, 1); - HTable t = new HTable(conf, TEST_TABLE.getTableName()); + Table t = new HTable(conf, TEST_TABLE.getTableName()); try { t.increment(inc); } finally { @@ -792,7 +791,7 @@ public class TestAccessController extends SecureTestUtil { public Object run() throws Exception { Delete d = new Delete(TEST_ROW); d.deleteFamily(TEST_FAMILY); - HTable t = new HTable(conf, TEST_TABLE.getTableName()); + Table t = new HTable(conf, TEST_TABLE.getTableName()); try { t.checkAndDelete(TEST_ROW, TEST_FAMILY, TEST_QUALIFIER, Bytes.toBytes("test_value"), d); @@ -810,7 +809,7 @@ public class TestAccessController extends SecureTestUtil { public Object run() throws Exception { Put p = new Put(TEST_ROW); p.add(TEST_FAMILY, TEST_QUALIFIER, Bytes.toBytes(1)); - HTable t = new HTable(conf, TEST_TABLE.getTableName()); + Table t = new HTable(conf, TEST_TABLE.getTableName()); try { t.checkAndPut(TEST_ROW, TEST_FAMILY, TEST_QUALIFIER, Bytes.toBytes("test_value"), p); @@ -948,7 +947,7 @@ public class TestAccessController extends SecureTestUtil { put.add(TEST_FAMILY, qualifier, Bytes.toBytes(1)); Append append = new Append(row); append.add(TEST_FAMILY, qualifier, Bytes.toBytes(2)); - HTable t = new HTable(conf, TEST_TABLE.getTableName()); + Table t = new HTable(conf, TEST_TABLE.getTableName()); try { t.put(put); t.append(append); @@ -968,7 +967,7 @@ public class TestAccessController extends SecureTestUtil { AccessTestAction grantAction = new AccessTestAction() { @Override public Object run() throws Exception { - HTable acl = new HTable(conf, AccessControlLists.ACL_TABLE_NAME); + Table acl = new HTable(conf, AccessControlLists.ACL_TABLE_NAME); try { BlockingRpcChannel service = acl.coprocessorService(TEST_TABLE.getTableName().getName()); AccessControlService.BlockingInterface protocol = @@ -985,7 +984,7 @@ public class TestAccessController extends SecureTestUtil { AccessTestAction revokeAction = new AccessTestAction() { @Override public Object run() throws Exception { - HTable acl = new HTable(conf, AccessControlLists.ACL_TABLE_NAME); + Table acl = new HTable(conf, AccessControlLists.ACL_TABLE_NAME); try { BlockingRpcChannel service = acl.coprocessorService(TEST_TABLE.getTableName().getName()); AccessControlService.BlockingInterface protocol = @@ -1002,7 +1001,7 @@ public class TestAccessController extends SecureTestUtil { AccessTestAction getPermissionsAction = new AccessTestAction() { @Override public Object run() throws Exception { - HTable acl = new HTable(conf, AccessControlLists.ACL_TABLE_NAME); + Table acl = new HTable(conf, AccessControlLists.ACL_TABLE_NAME); try { BlockingRpcChannel service = acl.coprocessorService(TEST_TABLE.getTableName().getName()); AccessControlService.BlockingInterface protocol = @@ -1057,7 +1056,7 @@ public class TestAccessController extends SecureTestUtil { Put p = new Put(Bytes.toBytes("a")); p.add(family1, qualifier, Bytes.toBytes("v1")); p.add(family2, qualifier, Bytes.toBytes("v2")); - HTable t = new HTable(conf, tableName); + Table t = new HTable(conf, tableName); try { t.put(p); } finally { @@ -1072,7 +1071,7 @@ public class TestAccessController extends SecureTestUtil { public Object run() throws Exception { Put p = new Put(Bytes.toBytes("a")); p.add(family1, qualifier, Bytes.toBytes("v1")); - HTable t = new HTable(conf, tableName); + Table t = new HTable(conf, tableName); try { t.put(p); } finally { @@ -1087,7 +1086,7 @@ public class TestAccessController extends SecureTestUtil { public Object run() throws Exception { Put p = new Put(Bytes.toBytes("a")); p.add(family2, qualifier, Bytes.toBytes("v2")); - HTable t = new HTable(conf, tableName); + Table t = new HTable(conf, tableName); try { t.put(p); } finally { @@ -1103,7 +1102,7 @@ public class TestAccessController extends SecureTestUtil { Get g = new Get(TEST_ROW); g.addFamily(family1); g.addFamily(family2); - HTable t = new HTable(conf, tableName); + Table t = new HTable(conf, tableName); try { t.get(g); } finally { @@ -1118,7 +1117,7 @@ public class TestAccessController extends SecureTestUtil { public Object run() throws Exception { Get g = new Get(TEST_ROW); g.addFamily(family1); - HTable t = new HTable(conf, tableName); + Table t = new HTable(conf, tableName); try { t.get(g); } finally { @@ -1133,7 +1132,7 @@ public class TestAccessController extends SecureTestUtil { public Object run() throws Exception { Get g = new Get(TEST_ROW); g.addFamily(family2); - HTable t = new HTable(conf, tableName); + Table t = new HTable(conf, tableName); try { t.get(g); } finally { @@ -1149,7 +1148,7 @@ public class TestAccessController extends SecureTestUtil { Delete d = new Delete(TEST_ROW); d.deleteFamily(family1); d.deleteFamily(family2); - HTable t = new HTable(conf, tableName); + Table t = new HTable(conf, tableName); try { t.delete(d); } finally { @@ -1164,7 +1163,7 @@ public class TestAccessController extends SecureTestUtil { public Object run() throws Exception { Delete d = new Delete(TEST_ROW); d.deleteFamily(family1); - HTable t = new HTable(conf, tableName); + Table t = new HTable(conf, tableName); try { t.delete(d); } finally { @@ -1179,7 +1178,7 @@ public class TestAccessController extends SecureTestUtil { public Object run() throws Exception { Delete d = new Delete(TEST_ROW); d.deleteFamily(family2); - HTable t = new HTable(conf, tableName); + Table t = new HTable(conf, tableName); try { t.delete(d); } finally { @@ -1326,7 +1325,7 @@ public class TestAccessController extends SecureTestUtil { public Object run() throws Exception { Get g = new Get(TEST_ROW); g.addColumn(family1, qualifier); - HTable t = new HTable(conf, tableName); + Table t = new HTable(conf, tableName); try { t.get(g); } finally { @@ -1341,7 +1340,7 @@ public class TestAccessController extends SecureTestUtil { public Object run() throws Exception { Put p = new Put(TEST_ROW); p.add(family1, qualifier, Bytes.toBytes("v1")); - HTable t = new HTable(conf, tableName); + Table t = new HTable(conf, tableName); try { t.put(p); } finally { @@ -1357,7 +1356,7 @@ public class TestAccessController extends SecureTestUtil { Delete d = new Delete(TEST_ROW); d.deleteColumn(family1, qualifier); // d.deleteFamily(family1); - HTable t = new HTable(conf, tableName); + Table t = new HTable(conf, tableName); try { t.delete(d); } finally { @@ -1435,7 +1434,7 @@ public class TestAccessController extends SecureTestUtil { List perms; - HTable acl = new HTable(conf, AccessControlLists.ACL_TABLE_NAME); + Table acl = new HTable(conf, AccessControlLists.ACL_TABLE_NAME); try { BlockingRpcChannel service = acl.coprocessorService(tableName.getName()); AccessControlService.BlockingInterface protocol = @@ -1548,7 +1547,7 @@ public class TestAccessController extends SecureTestUtil { @Test public void testGlobalPermissionList() throws Exception { List perms; - HTable acl = new HTable(conf, AccessControlLists.ACL_TABLE_NAME); + Table acl = new HTable(conf, AccessControlLists.ACL_TABLE_NAME); try { BlockingRpcChannel service = acl.coprocessorService(HConstants.EMPTY_START_ROW); AccessControlService.BlockingInterface protocol = @@ -1583,7 +1582,7 @@ public class TestAccessController extends SecureTestUtil { AccessControlProtos.GlobalPermission.newBuilder() .addAction(ProtobufUtil.toPermissionAction(a)).build())); } - HTable acl = new HTable(conf, AccessControlLists.ACL_TABLE_NAME); + Table acl = new HTable(conf, AccessControlLists.ACL_TABLE_NAME); try { BlockingRpcChannel channel = acl.coprocessorService(new byte[0]); AccessControlService.BlockingInterface protocol = @@ -1613,7 +1612,7 @@ public class TestAccessController extends SecureTestUtil { for (Permission p : perms) { request.addPermission(ProtobufUtil.toPermission(p)); } - HTable acl = new HTable(conf, table); + Table acl = new HTable(conf, table); try { AccessControlService.BlockingInterface protocol = AccessControlService.newBlockingStub(acl.coprocessorService(new byte[0])); @@ -1763,7 +1762,7 @@ public class TestAccessController extends SecureTestUtil { .setTableName(ProtobufUtil.toProtoTableName(TEST_TABLE.getTableName())) .addAction(AccessControlProtos.Permission.Action.CREATE)) ).build(); - HTable acl = new HTable(conf, AccessControlLists.ACL_TABLE_NAME); + Table acl = new HTable(conf, AccessControlLists.ACL_TABLE_NAME); try { BlockingRpcChannel channel = acl.coprocessorService(new byte[0]); AccessControlService.BlockingInterface protocol = @@ -2022,7 +2021,7 @@ public class TestAccessController extends SecureTestUtil { AccessTestAction getAction = new AccessTestAction() { @Override public Object run() throws Exception { - HTable t = new HTable(conf, TEST_TABLE.getTableName()); + Table t = new HTable(conf, TEST_TABLE.getTableName()); try { return t.get(new Get(TEST_ROW)); } finally { @@ -2112,7 +2111,7 @@ public class TestAccessController extends SecureTestUtil { AccessTestAction execEndpointAction = new AccessTestAction() { @Override public Object run() throws Exception { - HTable t = new HTable(conf, TEST_TABLE.getTableName()); + Table t = new HTable(conf, TEST_TABLE.getTableName()); try { BlockingRpcChannel service = t.coprocessorService(HConstants.EMPTY_BYTE_ARRAY); PingCoprocessor.newBlockingStub(service).noop(null, NoopRequest.newBuilder().build()); @@ -2141,7 +2140,7 @@ public class TestAccessController extends SecureTestUtil { AccessTestAction putWithReservedTag = new AccessTestAction() { @Override public Object run() throws Exception { - HTable t = new HTable(conf, TEST_TABLE.getTableName()); + Table t = new HTable(conf, TEST_TABLE.getTableName()); try { KeyValue kv = new KeyValue(TEST_ROW, TEST_FAMILY, TEST_QUALIFIER, HConstants.LATEST_TIMESTAMP, HConstants.EMPTY_BYTE_ARRAY, diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/security/access/TestAccessController2.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/security/access/TestAccessController2.java index 864739cd834..dcc3d16211b 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/security/access/TestAccessController2.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/security/access/TestAccessController2.java @@ -26,6 +26,7 @@ import org.apache.hadoop.hbase.HBaseTestingUtility; import org.apache.hadoop.hbase.HColumnDescriptor; import org.apache.hadoop.hbase.HTableDescriptor; import org.apache.hadoop.hbase.LargeTests; +import org.apache.hadoop.hbase.client.Admin; import org.apache.hadoop.hbase.client.HBaseAdmin; import org.apache.hadoop.hbase.security.User; import org.apache.hadoop.hbase.security.access.Permission.Action; @@ -75,7 +76,7 @@ public class TestAccessController2 extends SecureTestUtil { public Object run() throws Exception { HTableDescriptor desc = new HTableDescriptor(TEST_TABLE.getTableName()); desc.addFamily(new HColumnDescriptor(TEST_FAMILY)); - HBaseAdmin admin = new HBaseAdmin(conf); + Admin admin = new HBaseAdmin(conf); try { admin.createTable(desc); } finally { diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/security/access/TestCellACLWithMultipleVersions.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/security/access/TestCellACLWithMultipleVersions.java index fd8c9a1d403..40e2a02ad3c 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/security/access/TestCellACLWithMultipleVersions.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/security/access/TestCellACLWithMultipleVersions.java @@ -36,10 +36,10 @@ import org.apache.hadoop.hbase.TableNotFoundException; import org.apache.hadoop.hbase.client.Admin; import org.apache.hadoop.hbase.client.Delete; import org.apache.hadoop.hbase.client.Get; -import org.apache.hadoop.hbase.client.HBaseAdmin; import org.apache.hadoop.hbase.client.HTable; import org.apache.hadoop.hbase.client.Increment; import org.apache.hadoop.hbase.client.Put; +import org.apache.hadoop.hbase.client.Table; import org.apache.hadoop.hbase.master.MasterCoprocessorHost; import org.apache.hadoop.hbase.regionserver.RegionServerCoprocessorHost; import org.apache.hadoop.hbase.security.User; @@ -145,7 +145,7 @@ public class TestCellACLWithMultipleVersions extends SecureTestUtil { verifyAllowed(new AccessTestAction() { @Override public Object run() throws Exception { - HTable t = new HTable(conf, TEST_TABLE.getTableName()); + Table t = new HTable(conf, TEST_TABLE.getTableName()); try { Put p; // with ro ACL @@ -179,7 +179,7 @@ public class TestCellACLWithMultipleVersions extends SecureTestUtil { public Object run() throws Exception { Get get = new Get(TEST_ROW); get.setMaxVersions(10); - HTable t = new HTable(conf, TEST_TABLE.getTableName()); + Table t = new HTable(conf, TEST_TABLE.getTableName()); try { return t.get(get).listCells(); } finally { @@ -193,7 +193,7 @@ public class TestCellACLWithMultipleVersions extends SecureTestUtil { public Object run() throws Exception { Get get = new Get(TEST_ROW); get.setMaxVersions(10); - HTable t = new HTable(conf, TEST_TABLE.getTableName()); + Table t = new HTable(conf, TEST_TABLE.getTableName()); try { return t.get(get).listCells(); } finally { @@ -210,7 +210,7 @@ public class TestCellACLWithMultipleVersions extends SecureTestUtil { verifyAllowed(new AccessTestAction() { @Override public Object run() throws Exception { - HTable t = new HTable(conf, TEST_TABLE.getTableName()); + Table t = new HTable(conf, TEST_TABLE.getTableName()); try { Put p; p = new Put(TEST_ROW).add(TEST_FAMILY1, TEST_Q1, ZERO); @@ -249,7 +249,7 @@ public class TestCellACLWithMultipleVersions extends SecureTestUtil { verifyAllowed(new AccessTestAction() { @Override public Object run() throws Exception { - HTable t = new HTable(conf, TEST_TABLE.getTableName()); + Table t = new HTable(conf, TEST_TABLE.getTableName()); try { // with rw ACL for "user1" Put p = new Put(TEST_ROW1); @@ -275,7 +275,7 @@ public class TestCellACLWithMultipleVersions extends SecureTestUtil { verifyAllowed(new AccessTestAction() { @Override public Object run() throws Exception { - HTable t = new HTable(conf, TEST_TABLE.getTableName()); + Table t = new HTable(conf, TEST_TABLE.getTableName()); try { // with rw ACL for "user1" and "user2" Put p = new Put(TEST_ROW1); @@ -306,7 +306,7 @@ public class TestCellACLWithMultipleVersions extends SecureTestUtil { user1.runAs(new PrivilegedExceptionAction() { @Override public Void run() throws Exception { - HTable t = new HTable(conf, TEST_TABLE.getTableName()); + Table t = new HTable(conf, TEST_TABLE.getTableName()); try { Delete d = new Delete(TEST_ROW1); d.deleteColumns(TEST_FAMILY1, TEST_Q1); @@ -323,7 +323,7 @@ public class TestCellACLWithMultipleVersions extends SecureTestUtil { user2.runAs(new PrivilegedExceptionAction() { @Override public Void run() throws Exception { - HTable t = new HTable(conf, TEST_TABLE.getTableName()); + Table t = new HTable(conf, TEST_TABLE.getTableName()); try { Delete d = new Delete(TEST_ROW2); d.deleteColumns(TEST_FAMILY1, TEST_Q1); @@ -342,7 +342,7 @@ public class TestCellACLWithMultipleVersions extends SecureTestUtil { user1.runAs(new PrivilegedExceptionAction() { @Override public Void run() throws Exception { - HTable t = new HTable(conf, TEST_TABLE.getTableName()); + Table t = new HTable(conf, TEST_TABLE.getTableName()); try { Delete d = new Delete(TEST_ROW2); d.deleteFamily(TEST_FAMILY1); @@ -363,7 +363,7 @@ public class TestCellACLWithMultipleVersions extends SecureTestUtil { verifyAllowed(new AccessTestAction() { @Override public Object run() throws Exception { - HTable t = new HTable(conf, TEST_TABLE.getTableName()); + Table t = new HTable(conf, TEST_TABLE.getTableName()); try { // Store read only ACL at a future time Put p = new Put(TEST_ROW).add(TEST_FAMILY1, TEST_Q1, @@ -389,7 +389,7 @@ public class TestCellACLWithMultipleVersions extends SecureTestUtil { @Override public Object run() throws Exception { Get get = new Get(TEST_ROW).addColumn(TEST_FAMILY1, TEST_Q1); - HTable t = new HTable(conf, TEST_TABLE.getTableName()); + Table t = new HTable(conf, TEST_TABLE.getTableName()); try { return t.get(get).listCells(); } finally { @@ -402,7 +402,7 @@ public class TestCellACLWithMultipleVersions extends SecureTestUtil { @Override public Object run() throws Exception { Get get = new Get(TEST_ROW).addColumn(TEST_FAMILY1, TEST_Q2); - HTable t = new HTable(conf, TEST_TABLE.getTableName()); + Table t = new HTable(conf, TEST_TABLE.getTableName()); try { return t.get(get).listCells(); } finally { @@ -422,7 +422,7 @@ public class TestCellACLWithMultipleVersions extends SecureTestUtil { @Override public Object run() throws Exception { Delete delete = new Delete(TEST_ROW).deleteFamily(TEST_FAMILY1); - HTable t = new HTable(conf, TEST_TABLE.getTableName()); + Table t = new HTable(conf, TEST_TABLE.getTableName()); try { t.delete(delete); } finally { @@ -448,7 +448,7 @@ public class TestCellACLWithMultipleVersions extends SecureTestUtil { USER_OWNER.runAs(new AccessTestAction() { @Override public Object run() throws Exception { - HTable t = new HTable(conf, TEST_TABLE.getTableName()); + Table t = new HTable(conf, TEST_TABLE.getTableName()); try { // This version (TS = 123) with rw ACL for USER_OTHER and USER_OTHER2 Put p = new Put(TEST_ROW); @@ -493,7 +493,7 @@ public class TestCellACLWithMultipleVersions extends SecureTestUtil { USER_OTHER2.runAs(new AccessTestAction() { @Override public Object run() throws Exception { - HTable t = new HTable(conf, TEST_TABLE.getTableName()); + Table t = new HTable(conf, TEST_TABLE.getTableName()); try { Delete d = new Delete(TEST_ROW, 124L); d.deleteColumns(TEST_FAMILY1, TEST_Q1); @@ -509,7 +509,7 @@ public class TestCellACLWithMultipleVersions extends SecureTestUtil { USER_OTHER2.runAs(new AccessTestAction() { @Override public Object run() throws Exception { - HTable t = new HTable(conf, TEST_TABLE.getTableName()); + Table t = new HTable(conf, TEST_TABLE.getTableName()); try { Delete d = new Delete(TEST_ROW); d.deleteColumns(TEST_FAMILY1, TEST_Q2, 124L); @@ -535,7 +535,7 @@ public class TestCellACLWithMultipleVersions extends SecureTestUtil { verifyAllowed(new AccessTestAction() { @Override public Object run() throws Exception { - HTable t = new HTable(conf, TEST_TABLE.getTableName()); + Table t = new HTable(conf, TEST_TABLE.getTableName()); try { Map permsU1andOwner = new HashMap(); permsU1andOwner.put(user1.getShortName(), new Permission(Permission.Action.READ, @@ -592,7 +592,7 @@ public class TestCellACLWithMultipleVersions extends SecureTestUtil { user1.runAs(new PrivilegedExceptionAction() { @Override public Void run() throws Exception { - HTable t = new HTable(conf, TEST_TABLE.getTableName()); + Table t = new HTable(conf, TEST_TABLE.getTableName()); try { Delete d = new Delete(TEST_ROW1); d.deleteColumn(TEST_FAMILY1, TEST_Q1, 123); @@ -609,7 +609,7 @@ public class TestCellACLWithMultipleVersions extends SecureTestUtil { user2.runAs(new PrivilegedExceptionAction() { @Override public Void run() throws Exception { - HTable t = new HTable(conf, TEST_TABLE.getTableName()); + Table t = new HTable(conf, TEST_TABLE.getTableName()); try { Delete d = new Delete(TEST_ROW1, 127); d.deleteColumns(TEST_FAMILY1, TEST_Q1); @@ -640,7 +640,7 @@ public class TestCellACLWithMultipleVersions extends SecureTestUtil { verifyAllowed(new AccessTestAction() { @Override public Object run() throws Exception { - HTable t = new HTable(conf, TEST_TABLE.getTableName()); + Table t = new HTable(conf, TEST_TABLE.getTableName()); try { Map permsU1andOwner = new HashMap(); permsU1andOwner.put(user1.getShortName(), new Permission(Permission.Action.READ, @@ -680,7 +680,7 @@ public class TestCellACLWithMultipleVersions extends SecureTestUtil { user1.runAs(new PrivilegedExceptionAction() { @Override public Void run() throws Exception { - HTable t = new HTable(conf, TEST_TABLE.getTableName()); + Table t = new HTable(conf, TEST_TABLE.getTableName()); try { Increment inc = new Increment(TEST_ROW1); inc.setTimeRange(0, 123); @@ -697,7 +697,7 @@ public class TestCellACLWithMultipleVersions extends SecureTestUtil { user2.runAs(new PrivilegedExceptionAction() { @Override public Void run() throws Exception { - HTable t = new HTable(conf, TEST_TABLE.getTableName()); + Table t = new HTable(conf, TEST_TABLE.getTableName()); try { Increment inc = new Increment(TEST_ROW1); inc.setTimeRange(0, 127); @@ -727,7 +727,7 @@ public class TestCellACLWithMultipleVersions extends SecureTestUtil { verifyAllowed(new AccessTestAction() { @Override public Object run() throws Exception { - HTable t = new HTable(conf, TEST_TABLE.getTableName()); + Table t = new HTable(conf, TEST_TABLE.getTableName()); try { Map permsU1andOwner = new HashMap(); permsU1andOwner.put(user1.getShortName(), new Permission(Permission.Action.READ, @@ -769,7 +769,7 @@ public class TestCellACLWithMultipleVersions extends SecureTestUtil { user1.runAs(new PrivilegedExceptionAction() { @Override public Void run() throws Exception { - HTable t = new HTable(conf, TEST_TABLE.getTableName()); + Table t = new HTable(conf, TEST_TABLE.getTableName()); try { Put p = new Put(TEST_ROW1); p.add(TEST_FAMILY1, TEST_Q1, 125, ZERO); @@ -788,7 +788,7 @@ public class TestCellACLWithMultipleVersions extends SecureTestUtil { user2.runAs(new PrivilegedExceptionAction() { @Override public Void run() throws Exception { - HTable t = new HTable(conf, TEST_TABLE.getTableName()); + Table t = new HTable(conf, TEST_TABLE.getTableName()); try { Put p = new Put(TEST_ROW1); // column Q1 covers version at 123 fr which user2 do not have permission @@ -817,7 +817,7 @@ public class TestCellACLWithMultipleVersions extends SecureTestUtil { verifyAllowed(new AccessTestAction() { @Override public Object run() throws Exception { - HTable t = new HTable(conf, TEST_TABLE.getTableName()); + Table t = new HTable(conf, TEST_TABLE.getTableName()); try { Map permsU1andOwner = new HashMap(); permsU1andOwner.put(user1.getShortName(), new Permission(Permission.Action.READ, @@ -870,7 +870,7 @@ public class TestCellACLWithMultipleVersions extends SecureTestUtil { user1.runAs(new PrivilegedExceptionAction() { @Override public Void run() throws Exception { - HTable t = new HTable(conf, TEST_TABLE.getTableName()); + Table t = new HTable(conf, TEST_TABLE.getTableName()); try { Delete d = new Delete(TEST_ROW1); d.deleteColumns(TEST_FAMILY1, TEST_Q1, 120); @@ -886,7 +886,7 @@ public class TestCellACLWithMultipleVersions extends SecureTestUtil { user2.runAs(new PrivilegedExceptionAction() { @Override public Void run() throws Exception { - HTable t = new HTable(conf, TEST_TABLE.getTableName()); + Table t = new HTable(conf, TEST_TABLE.getTableName()); try { Delete d = new Delete(TEST_ROW1); d.deleteColumns(TEST_FAMILY1, TEST_Q1); @@ -905,7 +905,7 @@ public class TestCellACLWithMultipleVersions extends SecureTestUtil { user2.runAs(new PrivilegedExceptionAction() { @Override public Void run() throws Exception { - HTable t = new HTable(conf, TEST_TABLE.getTableName()); + Table t = new HTable(conf, TEST_TABLE.getTableName()); try { Delete d = new Delete(TEST_ROW1); d.deleteColumn(TEST_FAMILY1, TEST_Q2, 120); diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/security/access/TestCellACLs.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/security/access/TestCellACLs.java index 555986ca7d9..f58847ee37d 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/security/access/TestCellACLs.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/security/access/TestCellACLs.java @@ -34,13 +34,13 @@ import org.apache.hadoop.hbase.TableNotFoundException; import org.apache.hadoop.hbase.client.Admin; import org.apache.hadoop.hbase.client.Delete; import org.apache.hadoop.hbase.client.Get; -import org.apache.hadoop.hbase.client.HBaseAdmin; import org.apache.hadoop.hbase.client.HTable; import org.apache.hadoop.hbase.client.Increment; import org.apache.hadoop.hbase.client.Put; import org.apache.hadoop.hbase.client.Result; import org.apache.hadoop.hbase.client.ResultScanner; import org.apache.hadoop.hbase.client.Scan; +import org.apache.hadoop.hbase.client.Table; import org.apache.hadoop.hbase.master.MasterCoprocessorHost; import org.apache.hadoop.hbase.regionserver.RegionServerCoprocessorHost; import org.apache.hadoop.hbase.security.User; @@ -141,7 +141,7 @@ public class TestCellACLs extends SecureTestUtil { verifyAllowed(new AccessTestAction() { @Override public Object run() throws Exception { - HTable t = new HTable(conf, TEST_TABLE.getTableName()); + Table t = new HTable(conf, TEST_TABLE.getTableName()); try { Put p; // with ro ACL @@ -170,7 +170,7 @@ public class TestCellACLs extends SecureTestUtil { @Override public Object run() throws Exception { Get get = new Get(TEST_ROW).addColumn(TEST_FAMILY, TEST_Q1); - HTable t = new HTable(conf, TEST_TABLE.getTableName()); + Table t = new HTable(conf, TEST_TABLE.getTableName()); try { return t.get(get).listCells(); } finally { @@ -183,7 +183,7 @@ public class TestCellACLs extends SecureTestUtil { @Override public Object run() throws Exception { Get get = new Get(TEST_ROW).addColumn(TEST_FAMILY, TEST_Q2); - HTable t = new HTable(conf, TEST_TABLE.getTableName()); + Table t = new HTable(conf, TEST_TABLE.getTableName()); try { return t.get(get).listCells(); } finally { @@ -196,7 +196,7 @@ public class TestCellACLs extends SecureTestUtil { @Override public Object run() throws Exception { Get get = new Get(TEST_ROW).addColumn(TEST_FAMILY, TEST_Q3); - HTable t = new HTable(conf, TEST_TABLE.getTableName()); + Table t = new HTable(conf, TEST_TABLE.getTableName()); try { return t.get(get).listCells(); } finally { @@ -209,7 +209,7 @@ public class TestCellACLs extends SecureTestUtil { @Override public Object run() throws Exception { Get get = new Get(TEST_ROW).addColumn(TEST_FAMILY, TEST_Q4); - HTable t = new HTable(conf, TEST_TABLE.getTableName()); + Table t = new HTable(conf, TEST_TABLE.getTableName()); try { return t.get(get).listCells(); } finally { @@ -241,7 +241,7 @@ public class TestCellACLs extends SecureTestUtil { scan.setStartRow(TEST_ROW); scan.setStopRow(Bytes.add(TEST_ROW, new byte[]{ 0 } )); scan.addFamily(TEST_FAMILY); - HTable t = new HTable(conf, TEST_TABLE.getTableName()); + Table t = new HTable(conf, TEST_TABLE.getTableName()); try { ResultScanner scanner = t.getScanner(scan); Result result = null; @@ -274,7 +274,7 @@ public class TestCellACLs extends SecureTestUtil { @Override public Object run() throws Exception { Increment i = new Increment(TEST_ROW).addColumn(TEST_FAMILY, TEST_Q1, 1L); - HTable t = new HTable(conf, TEST_TABLE.getTableName()); + Table t = new HTable(conf, TEST_TABLE.getTableName()); try { t.increment(i); } finally { @@ -288,7 +288,7 @@ public class TestCellACLs extends SecureTestUtil { @Override public Object run() throws Exception { Increment i = new Increment(TEST_ROW).addColumn(TEST_FAMILY, TEST_Q2, 1L); - HTable t = new HTable(conf, TEST_TABLE.getTableName()); + Table t = new HTable(conf, TEST_TABLE.getTableName()); try { t.increment(i); } finally { @@ -304,7 +304,7 @@ public class TestCellACLs extends SecureTestUtil { Increment i = new Increment(TEST_ROW).addColumn(TEST_FAMILY, TEST_Q2, 1L); // Tag this increment with an ACL that denies write permissions to USER_OTHER i.setACL(USER_OTHER.getShortName(), new Permission(Action.READ)); - HTable t = new HTable(conf, TEST_TABLE.getTableName()); + Table t = new HTable(conf, TEST_TABLE.getTableName()); try { t.increment(i); } finally { @@ -318,7 +318,7 @@ public class TestCellACLs extends SecureTestUtil { @Override public Object run() throws Exception { Increment i = new Increment(TEST_ROW).addColumn(TEST_FAMILY, TEST_Q3, 1L); - HTable t = new HTable(conf, TEST_TABLE.getTableName()); + Table t = new HTable(conf, TEST_TABLE.getTableName()); try { t.increment(i); } finally { @@ -345,7 +345,7 @@ public class TestCellACLs extends SecureTestUtil { @Override public Object run() throws Exception { Delete delete = new Delete(TEST_ROW).deleteFamily(TEST_FAMILY); - HTable t = new HTable(conf, TEST_TABLE.getTableName()); + Table t = new HTable(conf, TEST_TABLE.getTableName()); try { t.delete(delete); } finally { @@ -359,7 +359,7 @@ public class TestCellACLs extends SecureTestUtil { @Override public Object run() throws Exception { Delete delete = new Delete(TEST_ROW).deleteColumn(TEST_FAMILY, TEST_Q1); - HTable t = new HTable(conf, TEST_TABLE.getTableName()); + Table t = new HTable(conf, TEST_TABLE.getTableName()); try { t.delete(delete); } finally { @@ -390,7 +390,7 @@ public class TestCellACLs extends SecureTestUtil { verifyDenied(new AccessTestAction() { @Override public Object run() throws Exception { - HTable t = new HTable(conf, TEST_TABLE.getTableName()); + Table t = new HTable(conf, TEST_TABLE.getTableName()); try { Put p; p = new Put(TEST_ROW).add(TEST_FAMILY, TEST_Q1, ZERO); @@ -406,7 +406,7 @@ public class TestCellACLs extends SecureTestUtil { verifyAllowed(new AccessTestAction() { @Override public Object run() throws Exception { - HTable t = new HTable(conf, TEST_TABLE.getTableName()); + Table t = new HTable(conf, TEST_TABLE.getTableName()); try { Put p; p = new Put(TEST_ROW).add(TEST_FAMILY, TEST_Q1, ZERO); @@ -422,7 +422,7 @@ public class TestCellACLs extends SecureTestUtil { verifyDenied(new AccessTestAction() { @Override public Object run() throws Exception { - HTable t = new HTable(conf, TEST_TABLE.getTableName()); + Table t = new HTable(conf, TEST_TABLE.getTableName()); try { Put p; p = new Put(TEST_ROW).add(TEST_FAMILY, TEST_Q1, ONE); @@ -438,7 +438,7 @@ public class TestCellACLs extends SecureTestUtil { verifyAllowed(new AccessTestAction() { @Override public Object run() throws Exception { - HTable t = new HTable(conf, TEST_TABLE.getTableName()); + Table t = new HTable(conf, TEST_TABLE.getTableName()); try { return t.get(new Get(TEST_ROW).addColumn(TEST_FAMILY, TEST_Q1)); } finally { diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/security/access/TestNamespaceCommands.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/security/access/TestNamespaceCommands.java index 7525ff934ba..0990dd9c793 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/security/access/TestNamespaceCommands.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/security/access/TestNamespaceCommands.java @@ -30,6 +30,7 @@ import org.apache.hadoop.hbase.NamespaceDescriptor; import org.apache.hadoop.hbase.client.Get; import org.apache.hadoop.hbase.client.HTable; import org.apache.hadoop.hbase.client.Result; +import org.apache.hadoop.hbase.client.Table; import org.apache.hadoop.hbase.coprocessor.MasterCoprocessorEnvironment; import org.apache.hadoop.hbase.coprocessor.ObserverContext; import org.apache.hadoop.hbase.protobuf.ProtobufUtil; @@ -96,7 +97,7 @@ public class TestNamespaceCommands extends SecureTestUtil { @Test public void testAclTableEntries() throws Exception { String userTestNamespace = "userTestNsp"; - HTable acl = new HTable(conf, AccessControlLists.ACL_TABLE_NAME); + Table acl = new HTable(conf, AccessControlLists.ACL_TABLE_NAME); try { // Grant and check state in ACL table grantOnNamespace(UTIL, userTestNamespace, TestNamespace, @@ -151,7 +152,7 @@ public class TestNamespaceCommands extends SecureTestUtil { AccessTestAction grantAction = new AccessTestAction() { public Object run() throws Exception { - HTable acl = new HTable(conf, AccessControlLists.ACL_TABLE_NAME); + Table acl = new HTable(conf, AccessControlLists.ACL_TABLE_NAME); try { BlockingRpcChannel service = acl.coprocessorService(HConstants.EMPTY_START_ROW); @@ -167,7 +168,7 @@ public class TestNamespaceCommands extends SecureTestUtil { AccessTestAction revokeAction = new AccessTestAction() { public Object run() throws Exception { - HTable acl = new HTable(conf, AccessControlLists.ACL_TABLE_NAME); + Table acl = new HTable(conf, AccessControlLists.ACL_TABLE_NAME); try { BlockingRpcChannel service = acl.coprocessorService(HConstants.EMPTY_START_ROW); diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/security/access/TestScanEarlyTermination.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/security/access/TestScanEarlyTermination.java index 0a1397b24b8..d78e43ea7a7 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/security/access/TestScanEarlyTermination.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/security/access/TestScanEarlyTermination.java @@ -31,11 +31,11 @@ import org.apache.hadoop.hbase.HTableDescriptor; import org.apache.hadoop.hbase.MediumTests; 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.HTable; import org.apache.hadoop.hbase.client.Put; import org.apache.hadoop.hbase.client.Result; import org.apache.hadoop.hbase.client.Scan; +import org.apache.hadoop.hbase.client.Table; import org.apache.hadoop.hbase.master.MasterCoprocessorHost; import org.apache.hadoop.hbase.regionserver.RegionServerCoprocessorHost; import org.apache.hadoop.hbase.security.User; @@ -155,7 +155,7 @@ public class TestScanEarlyTermination extends SecureTestUtil { public Object run() throws Exception { // force a new RS connection conf.set("testkey", UUID.randomUUID().toString()); - HTable t = new HTable(conf, TEST_TABLE.getTableName()); + Table t = new HTable(conf, TEST_TABLE.getTableName()); try { Put put = new Put(TEST_ROW).add(TEST_FAMILY1, TEST_Q1, ZERO); t.put(put); @@ -180,7 +180,7 @@ public class TestScanEarlyTermination extends SecureTestUtil { public Object run() throws Exception { // force a new RS connection conf.set("testkey", UUID.randomUUID().toString()); - HTable t = new HTable(conf, TEST_TABLE.getTableName()); + Table t = new HTable(conf, TEST_TABLE.getTableName()); try { Scan scan = new Scan().addFamily(TEST_FAMILY1); Result result = t.getScanner(scan).next(); @@ -204,7 +204,7 @@ public class TestScanEarlyTermination extends SecureTestUtil { public Object run() throws Exception { // force a new RS connection conf.set("testkey", UUID.randomUUID().toString()); - HTable t = new HTable(conf, TEST_TABLE.getTableName()); + Table t = new HTable(conf, TEST_TABLE.getTableName()); try { Scan scan = new Scan(); Result result = t.getScanner(scan).next(); @@ -226,7 +226,7 @@ public class TestScanEarlyTermination extends SecureTestUtil { public Object run() throws Exception { // force a new RS connection conf.set("testkey", UUID.randomUUID().toString()); - HTable t = new HTable(conf, TEST_TABLE.getTableName()); + Table t = new HTable(conf, TEST_TABLE.getTableName()); try { Scan scan = new Scan().addFamily(TEST_FAMILY2); Result result = t.getScanner(scan).next(); @@ -252,7 +252,7 @@ public class TestScanEarlyTermination extends SecureTestUtil { public Object run() throws Exception { // force a new RS connection conf.set("testkey", UUID.randomUUID().toString()); - HTable t = new HTable(conf, TEST_TABLE.getTableName()); + Table t = new HTable(conf, TEST_TABLE.getTableName()); try { Scan scan = new Scan(); Result result = t.getScanner(scan).next(); diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/security/access/TestTablePermissions.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/security/access/TestTablePermissions.java index c43689a69c6..c65ebefaf2f 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/security/access/TestTablePermissions.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/security/access/TestTablePermissions.java @@ -40,6 +40,7 @@ import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.hbase.Abortable; import org.apache.hadoop.hbase.TableName; import org.apache.hadoop.hbase.client.Admin; +import org.apache.hadoop.hbase.client.Table; import org.apache.hadoop.hbase.exceptions.DeserializationException; import org.apache.hadoop.hbase.HBaseTestingUtility; import org.apache.hadoop.hbase.LargeTests; @@ -287,7 +288,7 @@ public class TestTablePermissions { ListMultimap preperms = AccessControlLists.getTablePermissions(conf, TEST_TABLE); - HTable table = new HTable(conf, TEST_TABLE); + Table table = new HTable(conf, TEST_TABLE); table.put(new Put(Bytes.toBytes("row1")) .add(TEST_FAMILY, TEST_QUALIFIER, Bytes.toBytes("v1"))); table.put(new Put(Bytes.toBytes("row2")) diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/security/visibility/ExpAsStringVisibilityLabelServiceImpl.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/security/visibility/ExpAsStringVisibilityLabelServiceImpl.java index 02d7ac38213..14083a9130b 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/security/visibility/ExpAsStringVisibilityLabelServiceImpl.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/security/visibility/ExpAsStringVisibilityLabelServiceImpl.java @@ -44,6 +44,7 @@ import org.apache.hadoop.hbase.client.Get; import org.apache.hadoop.hbase.client.HTable; import org.apache.hadoop.hbase.client.Put; import org.apache.hadoop.hbase.client.Result; +import org.apache.hadoop.hbase.client.Table; import org.apache.hadoop.hbase.coprocessor.RegionCoprocessorEnvironment; import org.apache.hadoop.hbase.regionserver.HRegion; import org.apache.hadoop.hbase.regionserver.OperationStatus; @@ -141,7 +142,7 @@ public class ExpAsStringVisibilityLabelServiceImpl implements VisibilityLabelSer Get get = new Get(user); List cells = null; if (labelsRegion == null) { - HTable table = null; + Table table = null; try { table = new HTable(conf, VisibilityConstants.LABELS_TABLE_NAME); Result result = table.get(get); diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/security/visibility/TestEnforcingScanLabelGenerator.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/security/visibility/TestEnforcingScanLabelGenerator.java index d4cbf4dd977..cec7517f102 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/security/visibility/TestEnforcingScanLabelGenerator.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/security/visibility/TestEnforcingScanLabelGenerator.java @@ -33,6 +33,7 @@ import org.apache.hadoop.hbase.client.Get; import org.apache.hadoop.hbase.client.HTable; import org.apache.hadoop.hbase.client.Put; import org.apache.hadoop.hbase.client.Result; +import org.apache.hadoop.hbase.client.Table; import org.apache.hadoop.hbase.security.User; import org.apache.hadoop.hbase.util.Bytes; import org.junit.AfterClass; @@ -97,7 +98,7 @@ public class TestEnforcingScanLabelGenerator { SUPERUSER.runAs(new PrivilegedExceptionAction() { public Void run() throws Exception { - HTable table = TEST_UTIL.createTable(tableName, CF); + Table table = TEST_UTIL.createTable(tableName, CF); try { Put put = new Put(ROW_1); put.add(CF, Q1, HConstants.LATEST_TIMESTAMP, value); @@ -119,7 +120,7 @@ public class TestEnforcingScanLabelGenerator { TESTUSER.runAs(new PrivilegedExceptionAction() { public Void run() throws Exception { - HTable table = new HTable(conf, tableName); + Table table = new HTable(conf, tableName); try { // Test that we enforce the defined set Get get = new Get(ROW_1); diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/security/visibility/TestVisibilityLabels.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/security/visibility/TestVisibilityLabels.java index 068b790ab3e..86845317c9c 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/security/visibility/TestVisibilityLabels.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/security/visibility/TestVisibilityLabels.java @@ -50,6 +50,7 @@ import org.apache.hadoop.hbase.client.Result; import org.apache.hadoop.hbase.client.ResultScanner; import org.apache.hadoop.hbase.client.RowMutations; import org.apache.hadoop.hbase.client.Scan; +import org.apache.hadoop.hbase.client.Table; import org.apache.hadoop.hbase.protobuf.generated.ClientProtos.RegionActionResult; import org.apache.hadoop.hbase.protobuf.generated.VisibilityLabelsProtos.GetAuthsResponse; import org.apache.hadoop.hbase.protobuf.generated.VisibilityLabelsProtos.VisibilityLabelsResponse; @@ -111,7 +112,7 @@ public abstract class TestVisibilityLabels { @Test public void testSimpleVisibilityLabels() throws Exception { TableName tableName = TableName.valueOf(TEST_NAME.getMethodName()); - HTable table = createTableAndWriteDataWithLabels(tableName, SECRET + "|" + CONFIDENTIAL, + Table table = createTableAndWriteDataWithLabels(tableName, SECRET + "|" + CONFIDENTIAL, PRIVATE + "|" + CONFIDENTIAL); try { Scan s = new Scan(); @@ -140,7 +141,7 @@ public abstract class TestVisibilityLabels { @Test public void testSimpleVisibilityLabelsWithUniCodeCharacters() throws Exception { TableName tableName = TableName.valueOf(TEST_NAME.getMethodName()); - HTable table = createTableAndWriteDataWithLabels(tableName, + Table table = createTableAndWriteDataWithLabels(tableName, SECRET + "|" + CellVisibility.quote(COPYRIGHT), "(" + CellVisibility.quote(COPYRIGHT) + "&" + CellVisibility.quote(ACCENT) + ")|" + CONFIDENTIAL, CellVisibility.quote(UNICODE_VIS_TAG) + "&" + SECRET); @@ -176,7 +177,7 @@ public abstract class TestVisibilityLabels { @Test public void testAuthorizationsWithSpecialUnicodeCharacters() throws Exception { TableName tableName = TableName.valueOf(TEST_NAME.getMethodName()); - HTable table = createTableAndWriteDataWithLabels(tableName, + Table table = createTableAndWriteDataWithLabels(tableName, CellVisibility.quote(UC1) + "|" + CellVisibility.quote(UC2), CellVisibility.quote(UC1), CellVisibility.quote(UNICODE_VIS_TAG)); try { @@ -211,7 +212,7 @@ public abstract class TestVisibilityLabels { @Test public void testVisibilityLabelsWithComplexLabels() throws Exception { TableName tableName = TableName.valueOf(TEST_NAME.getMethodName()); - HTable table = createTableAndWriteDataWithLabels(tableName, "(" + SECRET + "|" + CONFIDENTIAL + Table table = createTableAndWriteDataWithLabels(tableName, "(" + SECRET + "|" + CONFIDENTIAL + ")" + "&" + "!" + TOPSECRET, "(" + PRIVATE + "&" + CONFIDENTIAL + "&" + SECRET + ")", "(" + PRIVATE + "&" + CONFIDENTIAL + "&" + SECRET + ")", "(" + PRIVATE + "&" + CONFIDENTIAL + "&" + SECRET + ")"); @@ -246,7 +247,7 @@ public abstract class TestVisibilityLabels { @Test public void testVisibilityLabelsThatDoesNotPassTheCriteria() throws Exception { TableName tableName = TableName.valueOf(TEST_NAME.getMethodName()); - HTable table = createTableAndWriteDataWithLabels(tableName, "(" + SECRET + "|" + CONFIDENTIAL + Table table = createTableAndWriteDataWithLabels(tableName, "(" + SECRET + "|" + CONFIDENTIAL + ")", PRIVATE); try { Scan s = new Scan(); @@ -274,7 +275,7 @@ public abstract class TestVisibilityLabels { @Test public void testVisibilityLabelsInScanThatDoesNotMatchAnyDefinedLabels() throws Exception { TableName tableName = TableName.valueOf(TEST_NAME.getMethodName()); - HTable table = createTableAndWriteDataWithLabels(tableName, "(" + SECRET + "|" + CONFIDENTIAL + Table table = createTableAndWriteDataWithLabels(tableName, "(" + SECRET + "|" + CONFIDENTIAL + ")", PRIVATE); try { Scan s = new Scan(); @@ -292,7 +293,7 @@ public abstract class TestVisibilityLabels { @Test public void testVisibilityLabelsWithGet() throws Exception { TableName tableName = TableName.valueOf(TEST_NAME.getMethodName()); - HTable table = createTableAndWriteDataWithLabels(tableName, SECRET + "&" + CONFIDENTIAL + "&!" + Table table = createTableAndWriteDataWithLabels(tableName, SECRET + "&" + CONFIDENTIAL + "&!" + PRIVATE, SECRET + "&" + CONFIDENTIAL + "&" + PRIVATE); try { Get get = new Get(row1); @@ -375,7 +376,7 @@ public abstract class TestVisibilityLabels { } TEST_UTIL.waitTableEnabled(LABELS_TABLE_NAME.getName(), 50000); t.join(); - HTable table = null; + Table table = null; try { table = new HTable(TEST_UTIL.getConfiguration(), tableName); Scan s = new Scan(); @@ -393,7 +394,7 @@ public abstract class TestVisibilityLabels { @Test(timeout = 60 * 1000) public void testVisibilityLabelsOnRSRestart() throws Exception { final TableName tableName = TableName.valueOf(TEST_NAME.getMethodName()); - HTable table = createTableAndWriteDataWithLabels(tableName, "(" + SECRET + "|" + CONFIDENTIAL + Table table = createTableAndWriteDataWithLabels(tableName, "(" + SECRET + "|" + CONFIDENTIAL + ")", PRIVATE); List regionServerThreads = TEST_UTIL.getHBaseCluster() .getRegionServerThreads(); @@ -441,7 +442,7 @@ public abstract class TestVisibilityLabels { @Test public void testVisibilityLabelsInGetThatDoesNotMatchAnyDefinedLabels() throws Exception { TableName tableName = TableName.valueOf(TEST_NAME.getMethodName()); - HTable table = createTableAndWriteDataWithLabels(tableName, "(" + SECRET + "|" + CONFIDENTIAL + Table table = createTableAndWriteDataWithLabels(tableName, "(" + SECRET + "|" + CONFIDENTIAL + ")", PRIVATE); try { Get get = new Get(row1); @@ -469,7 +470,7 @@ public abstract class TestVisibilityLabels { } }; SUPERUSER.runAs(action); - HTable ht = null; + Table ht = null; try { ht = new HTable(conf, LABELS_TABLE_NAME); Scan scan = new Scan(); @@ -580,7 +581,7 @@ public abstract class TestVisibilityLabels { "org.apache.hadoop.hbase.security.visibility.InvalidLabelException: " + "Label 'public' is not set for the user testUser")); assertTrue(resultList.get(2).getException().getValue().isEmpty()); - HTable ht = null; + Table ht = null; try { ht = new HTable(conf, LABELS_TABLE_NAME); ResultScanner scanner = ht.getScanner(new Scan()); @@ -619,7 +620,7 @@ public abstract class TestVisibilityLabels { @Test public void testLabelsWithCheckAndPut() throws Throwable { TableName tableName = TableName.valueOf(TEST_NAME.getMethodName()); - HTable table = null; + Table table = null; try { table = TEST_UTIL.createTable(tableName, fam); byte[] row1 = Bytes.toBytes("row1"); @@ -651,7 +652,7 @@ public abstract class TestVisibilityLabels { @Test public void testLabelsWithIncrement() throws Throwable { TableName tableName = TableName.valueOf(TEST_NAME.getMethodName()); - HTable table = null; + Table table = null; try { table = TEST_UTIL.createTable(tableName, fam); byte[] row1 = Bytes.toBytes("row1"); @@ -683,7 +684,7 @@ public abstract class TestVisibilityLabels { @Test public void testLabelsWithAppend() throws Throwable { TableName tableName = TableName.valueOf(TEST_NAME.getMethodName()); - HTable table = null; + Table table = null; try { table = TEST_UTIL.createTable(tableName, fam); byte[] row1 = Bytes.toBytes("row1"); @@ -771,7 +772,7 @@ public abstract class TestVisibilityLabels { col.setMaxVersions(5); desc.addFamily(col); TEST_UTIL.getHBaseAdmin().createTable(desc); - HTable table = null; + Table table = null; try { table = new HTable(TEST_UTIL.getConfiguration(), tableName); Put put = new Put(r1); @@ -861,7 +862,7 @@ public abstract class TestVisibilityLabels { HColumnDescriptor col = new HColumnDescriptor(fam); desc.addFamily(col); TEST_UTIL.getHBaseAdmin().createTable(desc); - HTable table = new HTable(TEST_UTIL.getConfiguration(), tableName); + Table table = new HTable(TEST_UTIL.getConfiguration(), tableName); try { Put p1 = new Put(row1); p1.add(fam, qual, value); @@ -892,9 +893,9 @@ public abstract class TestVisibilityLabels { } } - static HTable createTableAndWriteDataWithLabels(TableName tableName, String... labelExps) + static Table createTableAndWriteDataWithLabels(TableName tableName, String... labelExps) throws Exception { - HTable table = null; + Table table = null; try { table = TEST_UTIL.createTable(tableName, fam); int i = 1; diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/security/visibility/TestVisibilityLabelsWithACL.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/security/visibility/TestVisibilityLabelsWithACL.java index 15b31363055..17693d20bf9 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/security/visibility/TestVisibilityLabelsWithACL.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/security/visibility/TestVisibilityLabelsWithACL.java @@ -38,6 +38,7 @@ import org.apache.hadoop.hbase.client.Put; import org.apache.hadoop.hbase.client.Result; import org.apache.hadoop.hbase.client.ResultScanner; import org.apache.hadoop.hbase.client.Scan; +import org.apache.hadoop.hbase.client.Table; import org.apache.hadoop.hbase.protobuf.generated.VisibilityLabelsProtos.GetAuthsResponse; import org.apache.hadoop.hbase.protobuf.generated.VisibilityLabelsProtos.VisibilityLabelsResponse; import org.apache.hadoop.hbase.security.User; @@ -122,7 +123,7 @@ public class TestVisibilityLabelsWithACL { public Void run() throws Exception { Scan s = new Scan(); s.setAuthorizations(new Authorizations(SECRET, CONFIDENTIAL)); - HTable t = new HTable(conf, table.getTableName()); + Table t = new HTable(conf, table.getTableName()); try { ResultScanner scanner = t.getScanner(s); Result result = scanner.next(); @@ -151,7 +152,7 @@ public class TestVisibilityLabelsWithACL { public Void run() throws Exception { Scan s = new Scan(); s.setAuthorizations(new Authorizations(SECRET, CONFIDENTIAL)); - HTable t = new HTable(conf, table.getTableName()); + Table t = new HTable(conf, table.getTableName()); try { ResultScanner scanner = t.getScanner(s); Result[] result = scanner.next(5); @@ -177,7 +178,7 @@ public class TestVisibilityLabelsWithACL { public Void run() throws Exception { Get g = new Get(row1); g.setAuthorizations(new Authorizations(SECRET, CONFIDENTIAL)); - HTable t = new HTable(conf, table.getTableName()); + Table t = new HTable(conf, table.getTableName()); try { Result result = t.get(g); assertTrue(!result.isEmpty()); @@ -206,7 +207,7 @@ public class TestVisibilityLabelsWithACL { public Void run() throws Exception { Get g = new Get(row1); g.setAuthorizations(new Authorizations(SECRET, CONFIDENTIAL)); - HTable t = new HTable(conf, table.getTableName()); + Table t = new HTable(conf, table.getTableName()); try { Result result = t.get(g); assertTrue(result.isEmpty()); diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/security/visibility/TestVisibilityLabelsWithDefaultVisLabelService.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/security/visibility/TestVisibilityLabelsWithDefaultVisLabelService.java index 3153d7906e2..9a0400348eb 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/security/visibility/TestVisibilityLabelsWithDefaultVisLabelService.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/security/visibility/TestVisibilityLabelsWithDefaultVisLabelService.java @@ -35,9 +35,9 @@ import org.apache.hadoop.hbase.client.HTable; import org.apache.hadoop.hbase.client.Result; import org.apache.hadoop.hbase.client.ResultScanner; import org.apache.hadoop.hbase.client.Scan; +import org.apache.hadoop.hbase.client.Table; import org.apache.hadoop.hbase.protobuf.ProtobufUtil; import org.apache.hadoop.hbase.protobuf.generated.ClientProtos.RegionActionResult; -import org.apache.hadoop.hbase.protobuf.generated.ClientProtos.ResultOrException; import org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.NameBytesPair; import org.apache.hadoop.hbase.protobuf.generated.VisibilityLabelsProtos.VisibilityLabelsResponse; import org.apache.hadoop.hbase.security.User; @@ -144,7 +144,7 @@ public class TestVisibilityLabelsWithDefaultVisLabelService extends TestVisibili // Scan the visibility label Scan s = new Scan(); s.setAuthorizations(new Authorizations(VisibilityUtils.SYSTEM_LABEL)); - HTable ht = new HTable(conf, LABELS_TABLE_NAME.getName()); + Table ht = new HTable(conf, LABELS_TABLE_NAME.getName()); int i = 0; try { ResultScanner scanner = ht.getScanner(s); diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/security/visibility/TestVisibilityLabelsWithDeletes.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/security/visibility/TestVisibilityLabelsWithDeletes.java index 8b27c19ae8e..1a617dc22c8 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/security/visibility/TestVisibilityLabelsWithDeletes.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/security/visibility/TestVisibilityLabelsWithDeletes.java @@ -44,6 +44,7 @@ import org.apache.hadoop.hbase.client.Result; import org.apache.hadoop.hbase.client.ResultScanner; import org.apache.hadoop.hbase.client.RetriesExhaustedWithDetailsException; import org.apache.hadoop.hbase.client.Scan; +import org.apache.hadoop.hbase.client.Table; import org.apache.hadoop.hbase.protobuf.generated.VisibilityLabelsProtos.VisibilityLabelsResponse; import org.apache.hadoop.hbase.security.User; import org.apache.hadoop.hbase.util.Bytes; @@ -110,13 +111,13 @@ public class TestVisibilityLabelsWithDeletes { public void testVisibilityLabelsWithDeleteColumns() throws Throwable { setAuths(); TableName tableName = TableName.valueOf(TEST_NAME.getMethodName()); - final HTable table = createTableAndWriteDataWithLabels(tableName, SECRET + "&" + TOPSECRET, + final Table table = createTableAndWriteDataWithLabels(tableName, SECRET + "&" + TOPSECRET, SECRET); try { PrivilegedExceptionAction actiona = new PrivilegedExceptionAction() { @Override public Void run() throws Exception { - HTable table = null; + Table table = null; try { table = new HTable(conf, TEST_NAME.getMethodName()); Delete d = new Delete(row1); @@ -156,14 +157,14 @@ public class TestVisibilityLabelsWithDeletes { public void testVisibilityLabelsWithDeleteFamily() throws Exception { setAuths(); TableName tableName = TableName.valueOf(TEST_NAME.getMethodName()); - final HTable table = createTableAndWriteDataWithLabels(tableName, SECRET, CONFIDENTIAL + "|" + final Table table = createTableAndWriteDataWithLabels(tableName, SECRET, CONFIDENTIAL + "|" + TOPSECRET); try { PrivilegedExceptionAction actiona = new PrivilegedExceptionAction() { @Override public Void run() throws Exception { try { - HTable table = new HTable(conf, TEST_NAME.getMethodName()); + Table table = new HTable(conf, TEST_NAME.getMethodName()); Delete d = new Delete(row2); d.setCellVisibility(new CellVisibility(TOPSECRET + "|" + CONFIDENTIAL)); d.deleteFamily(fam); @@ -199,13 +200,13 @@ public class TestVisibilityLabelsWithDeletes { setAuths(); TableName tableName = TableName.valueOf(TEST_NAME.getMethodName()); long[] ts = new long[] { 123l, 125l }; - final HTable table = createTableAndWriteDataWithLabels(tableName, ts, CONFIDENTIAL + "|" + final Table table = createTableAndWriteDataWithLabels(tableName, ts, CONFIDENTIAL + "|" + TOPSECRET, SECRET); try { PrivilegedExceptionAction actiona = new PrivilegedExceptionAction() { @Override public Void run() throws Exception { - HTable table = null; + Table table = null; try { table = new HTable(conf, TEST_NAME.getMethodName()); Delete d = new Delete(row1); @@ -245,13 +246,13 @@ public class TestVisibilityLabelsWithDeletes { setAuths(); TableName tableName = TableName.valueOf(TEST_NAME.getMethodName()); long[] ts = new long[] { 123l, 125l }; - final HTable table = createTableAndWriteDataWithLabels(tableName, ts, CONFIDENTIAL + "|" + final Table table = createTableAndWriteDataWithLabels(tableName, ts, CONFIDENTIAL + "|" + TOPSECRET, SECRET); try { PrivilegedExceptionAction actiona = new PrivilegedExceptionAction() { @Override public Void run() throws Exception { - HTable table = null; + Table table = null; try { table = new HTable(conf, TEST_NAME.getMethodName()); Delete d = new Delete(row1); @@ -290,7 +291,7 @@ public class TestVisibilityLabelsWithDeletes { public void testVisibilityLabelsWithDeleteColumnsWithMultipleVersions() throws Exception { setAuths(); TableName tableName = TableName.valueOf(TEST_NAME.getMethodName()); - HTable table = null; + Table table = null; try { table = doPuts(tableName); TEST_UTIL.getHBaseAdmin().flush(tableName); @@ -298,7 +299,7 @@ public class TestVisibilityLabelsWithDeletes { @Override public Void run() throws Exception { try { - HTable table = new HTable(conf, TEST_NAME.getMethodName()); + Table table = new HTable(conf, TEST_NAME.getMethodName()); Delete d = new Delete(row1); d.setCellVisibility(new CellVisibility("(" + PRIVATE + "&" + CONFIDENTIAL + ")|(" + SECRET + "&" + TOPSECRET+")")); @@ -352,7 +353,7 @@ public class TestVisibilityLabelsWithDeletes { throws Exception { setAuths(); TableName tableName = TableName.valueOf(TEST_NAME.getMethodName()); - HTable table = null; + Table table = null; try { table = doPuts(tableName); TEST_UTIL.getHBaseAdmin().flush(tableName); @@ -360,7 +361,7 @@ public class TestVisibilityLabelsWithDeletes { @Override public Void run() throws Exception { try { - HTable table = new HTable(conf, TEST_NAME.getMethodName()); + Table table = new HTable(conf, TEST_NAME.getMethodName()); Delete d = new Delete(row1); d.setCellVisibility(new CellVisibility(CONFIDENTIAL)); d.deleteColumns(fam, qual); @@ -409,7 +410,7 @@ public class TestVisibilityLabelsWithDeletes { throws Exception { setAuths(); TableName tableName = TableName.valueOf(TEST_NAME.getMethodName()); - HTable table = null; + Table table = null; try { table = doPuts(tableName); TEST_UTIL.getHBaseAdmin().flush(tableName); @@ -417,7 +418,7 @@ public class TestVisibilityLabelsWithDeletes { @Override public Void run() throws Exception { try { - HTable table = new HTable(conf, TEST_NAME.getMethodName()); + Table table = new HTable(conf, TEST_NAME.getMethodName()); Delete d = new Delete(row1); d.setCellVisibility(new CellVisibility(CONFIDENTIAL)); d.deleteColumns(fam, qual); @@ -470,7 +471,7 @@ public class TestVisibilityLabelsWithDeletes { throws Exception { setAuths(); TableName tableName = TableName.valueOf(TEST_NAME.getMethodName()); - HTable table = null; + Table table = null; try { table = doPuts(tableName); TEST_UTIL.getHBaseAdmin().flush(tableName); @@ -478,7 +479,7 @@ public class TestVisibilityLabelsWithDeletes { @Override public Void run() throws Exception { try { - HTable table = new HTable(conf, TEST_NAME.getMethodName()); + Table table = new HTable(conf, TEST_NAME.getMethodName()); Delete d = new Delete(row1); d.setCellVisibility(new CellVisibility(CONFIDENTIAL)); d.deleteFamily(fam); @@ -524,7 +525,7 @@ public class TestVisibilityLabelsWithDeletes { @Test public void testVisibilityLabelsWithDeleteFamilyWithPutsReAppearing() throws Exception { TableName tableName = TableName.valueOf(TEST_NAME.getMethodName()); - HTable table = null; + Table table = null; try { Admin hBaseAdmin = TEST_UTIL.getHBaseAdmin(); HColumnDescriptor colDesc = new HColumnDescriptor(fam); @@ -546,7 +547,7 @@ public class TestVisibilityLabelsWithDeletes { @Override public Void run() throws Exception { try { - HTable table = new HTable(conf, TEST_NAME.getMethodName()); + Table table = new HTable(conf, TEST_NAME.getMethodName()); Delete d = new Delete(row1); d.setCellVisibility(new CellVisibility(CONFIDENTIAL)); d.deleteFamily(fam); @@ -573,7 +574,7 @@ public class TestVisibilityLabelsWithDeletes { @Override public Void run() throws Exception { try { - HTable table = new HTable(conf, TEST_NAME.getMethodName()); + Table table = new HTable(conf, TEST_NAME.getMethodName()); Delete d = new Delete(row1); d.setCellVisibility(new CellVisibility(SECRET)); d.deleteFamily(fam); @@ -608,7 +609,7 @@ public class TestVisibilityLabelsWithDeletes { @Test public void testVisibilityLabelsWithDeleteColumnsWithPutsReAppearing() throws Exception { TableName tableName = TableName.valueOf(TEST_NAME.getMethodName()); - HTable table = null; + Table table = null; try { Admin hBaseAdmin = TEST_UTIL.getHBaseAdmin(); HColumnDescriptor colDesc = new HColumnDescriptor(fam); @@ -630,7 +631,7 @@ public class TestVisibilityLabelsWithDeletes { @Override public Void run() throws Exception { try { - HTable table = new HTable(conf, TEST_NAME.getMethodName()); + Table table = new HTable(conf, TEST_NAME.getMethodName()); Delete d = new Delete(row1); d.setCellVisibility(new CellVisibility(CONFIDENTIAL)); d.deleteColumns(fam, qual); @@ -657,7 +658,7 @@ public class TestVisibilityLabelsWithDeletes { @Override public Void run() throws Exception { try { - HTable table = new HTable(conf, TEST_NAME.getMethodName()); + Table table = new HTable(conf, TEST_NAME.getMethodName()); Delete d = new Delete(row1); d.setCellVisibility(new CellVisibility(SECRET)); d.deleteColumns(fam, qual); @@ -692,7 +693,7 @@ public class TestVisibilityLabelsWithDeletes { @Test public void testVisibilityCombinations() throws Exception { TableName tableName = TableName.valueOf(TEST_NAME.getMethodName()); - HTable table = null; + Table table = null; try { Admin hBaseAdmin = TEST_UTIL.getHBaseAdmin(); HColumnDescriptor colDesc = new HColumnDescriptor(fam); @@ -714,7 +715,7 @@ public class TestVisibilityLabelsWithDeletes { @Override public Void run() throws Exception { try { - HTable table = new HTable(conf, TEST_NAME.getMethodName()); + Table table = new HTable(conf, TEST_NAME.getMethodName()); Delete d = new Delete(row1); d.setCellVisibility(new CellVisibility(SECRET)); d.deleteColumns(fam, qual, 126l); @@ -749,7 +750,7 @@ public class TestVisibilityLabelsWithDeletes { public void testVisibilityLabelsWithDeleteColumnWithSpecificVersionWithPutsReAppearing() throws Exception { TableName tableName = TableName.valueOf(TEST_NAME.getMethodName()); - HTable table = null; + Table table = null; try { Admin hBaseAdmin = TEST_UTIL.getHBaseAdmin(); HColumnDescriptor colDesc = new HColumnDescriptor(fam); @@ -778,7 +779,7 @@ public class TestVisibilityLabelsWithDeletes { @Override public Void run() throws Exception { try { - HTable table = new HTable(conf, TEST_NAME.getMethodName()); + Table table = new HTable(conf, TEST_NAME.getMethodName()); Delete d = new Delete(row1); d.setCellVisibility(new CellVisibility(CONFIDENTIAL)); d.deleteColumn(fam, qual, 123l); @@ -816,7 +817,7 @@ public class TestVisibilityLabelsWithDeletes { throws Exception { setAuths(); TableName tableName = TableName.valueOf(TEST_NAME.getMethodName()); - HTable table = null; + Table table = null; try { table = doPuts(tableName); TEST_UTIL.getHBaseAdmin().flush(tableName); @@ -824,7 +825,7 @@ public class TestVisibilityLabelsWithDeletes { @Override public Void run() throws Exception { try { - HTable table = new HTable(conf, TEST_NAME.getMethodName()); + Table table = new HTable(conf, TEST_NAME.getMethodName()); Delete d = new Delete(row1); d.setCellVisibility(new CellVisibility(CONFIDENTIAL)); d.deleteFamily(fam); @@ -876,7 +877,7 @@ public class TestVisibilityLabelsWithDeletes { public void testDeleteFamilyAndDeleteColumnsWithAndWithoutVisibilityExp() throws Exception { setAuths(); TableName tableName = TableName.valueOf(TEST_NAME.getMethodName()); - HTable table = null; + Table table = null; try { table = doPuts(tableName); TEST_UTIL.getHBaseAdmin().flush(tableName); @@ -884,7 +885,7 @@ public class TestVisibilityLabelsWithDeletes { @Override public Void run() throws Exception { try { - HTable table = new HTable(conf, TEST_NAME.getMethodName()); + Table table = new HTable(conf, TEST_NAME.getMethodName()); Delete d = new Delete(row1); d.deleteFamily(fam); table.delete(d); @@ -940,9 +941,9 @@ public class TestVisibilityLabelsWithDeletes { } } - private HTable doPuts(TableName tableName) throws IOException, InterruptedIOException, + private Table doPuts(TableName tableName) throws IOException, InterruptedIOException, RetriesExhaustedWithDetailsException, InterruptedException { - HTable table; + Table table; Admin hBaseAdmin = TEST_UTIL.getHBaseAdmin(); HColumnDescriptor colDesc = new HColumnDescriptor(fam); colDesc.setMaxVersions(5); @@ -982,9 +983,9 @@ public class TestVisibilityLabelsWithDeletes { return table; } - private HTable doPutsWithDiffCols(TableName tableName) throws IOException, + private Table doPutsWithDiffCols(TableName tableName) throws IOException, InterruptedIOException, RetriesExhaustedWithDetailsException, InterruptedException { - HTable table; + Table table; Admin hBaseAdmin = TEST_UTIL.getHBaseAdmin(); HColumnDescriptor colDesc = new HColumnDescriptor(fam); colDesc.setMaxVersions(5); @@ -1017,9 +1018,9 @@ public class TestVisibilityLabelsWithDeletes { return table; } - private HTable doPutsWithoutVisibility(TableName tableName) throws IOException, + private Table doPutsWithoutVisibility(TableName tableName) throws IOException, InterruptedIOException, RetriesExhaustedWithDetailsException, InterruptedException { - HTable table; + Table table; Admin hBaseAdmin = TEST_UTIL.getHBaseAdmin(); HColumnDescriptor colDesc = new HColumnDescriptor(fam); colDesc.setMaxVersions(5); @@ -1055,7 +1056,7 @@ public class TestVisibilityLabelsWithDeletes { throws Exception { setAuths(); TableName tableName = TableName.valueOf(TEST_NAME.getMethodName()); - HTable table = null; + Table table = null; try { table = doPuts(tableName); TEST_UTIL.getHBaseAdmin().flush(tableName); @@ -1063,7 +1064,7 @@ public class TestVisibilityLabelsWithDeletes { @Override public Void run() throws Exception { try { - HTable table = new HTable(conf, TEST_NAME.getMethodName()); + Table table = new HTable(conf, TEST_NAME.getMethodName()); Delete d = new Delete(row1); d.setCellVisibility(new CellVisibility("(" + PRIVATE + "&" + CONFIDENTIAL + ")|(" + SECRET + "&" + TOPSECRET+")")); @@ -1126,7 +1127,7 @@ public class TestVisibilityLabelsWithDeletes { public void testDeleteColumnWithLatestTimeStampUsingMultipleVersions() throws Exception { setAuths(); TableName tableName = TableName.valueOf(TEST_NAME.getMethodName()); - HTable table = null; + Table table = null; try { table = doPuts(tableName); TEST_UTIL.getHBaseAdmin().flush(tableName); @@ -1134,7 +1135,7 @@ public class TestVisibilityLabelsWithDeletes { @Override public Void run() throws Exception { try { - HTable table = new HTable(conf, TEST_NAME.getMethodName()); + Table table = new HTable(conf, TEST_NAME.getMethodName()); Delete d = new Delete(row1); d.setCellVisibility(new CellVisibility(SECRET + "&" + TOPSECRET)); d.deleteColumn(fam, qual); @@ -1191,7 +1192,7 @@ public class TestVisibilityLabelsWithDeletes { public void testDeleteColumnWithLatestTimeStampWhenNoVersionMatches() throws Exception { setAuths(); TableName tableName = TableName.valueOf(TEST_NAME.getMethodName()); - HTable table = null; + Table table = null; try { table = doPuts(tableName); TEST_UTIL.getHBaseAdmin().flush(tableName); @@ -1203,7 +1204,7 @@ public class TestVisibilityLabelsWithDeletes { @Override public Void run() throws Exception { try { - HTable table = new HTable(conf, TEST_NAME.getMethodName()); + Table table = new HTable(conf, TEST_NAME.getMethodName()); Delete d = new Delete(row1); d.setCellVisibility(new CellVisibility(SECRET )); d.deleteColumn(fam, qual); @@ -1284,7 +1285,7 @@ public class TestVisibilityLabelsWithDeletes { throws Exception { setAuths(); TableName tableName = TableName.valueOf(TEST_NAME.getMethodName()); - HTable table = null; + Table table = null; try { table = doPuts(tableName); TEST_UTIL.getHBaseAdmin().flush(tableName); @@ -1292,7 +1293,7 @@ public class TestVisibilityLabelsWithDeletes { @Override public Void run() throws Exception { try { - HTable table = new HTable(conf, TEST_NAME.getMethodName()); + Table table = new HTable(conf, TEST_NAME.getMethodName()); Delete d = new Delete(row1); d.setCellVisibility(new CellVisibility(SECRET + "&" + TOPSECRET)); d.deleteColumn(fam, qual); @@ -1356,7 +1357,7 @@ public class TestVisibilityLabelsWithDeletes { public void testDeleteFamilyLatestTimeStampWithMulipleVersions() throws Exception { setAuths(); TableName tableName = TableName.valueOf(TEST_NAME.getMethodName()); - HTable table = null; + Table table = null; try { table = doPuts(tableName); TEST_UTIL.getHBaseAdmin().flush(tableName); @@ -1364,7 +1365,7 @@ public class TestVisibilityLabelsWithDeletes { @Override public Void run() throws Exception { try { - HTable table = new HTable(conf, TEST_NAME.getMethodName()); + Table table = new HTable(conf, TEST_NAME.getMethodName()); Delete d = new Delete(row1); d.setCellVisibility(new CellVisibility(SECRET + "&" + TOPSECRET)); d.deleteFamily(fam); @@ -1411,7 +1412,7 @@ public class TestVisibilityLabelsWithDeletes { public void testDeleteColumnswithMultipleColumnsWithMultipleVersions() throws Exception { setAuths(); TableName tableName = TableName.valueOf(TEST_NAME.getMethodName()); - HTable table = null; + Table table = null; try { table = doPutsWithDiffCols(tableName); TEST_UTIL.getHBaseAdmin().flush(tableName); @@ -1419,7 +1420,7 @@ public class TestVisibilityLabelsWithDeletes { @Override public Void run() throws Exception { try { - HTable table = new HTable(conf, TEST_NAME.getMethodName()); + Table table = new HTable(conf, TEST_NAME.getMethodName()); Delete d = new Delete(row1); d.setCellVisibility(new CellVisibility(SECRET + "&" + TOPSECRET)); d.deleteColumns(fam, qual, 125l); @@ -1474,7 +1475,7 @@ public class TestVisibilityLabelsWithDeletes { @Test public void testDeleteColumnsWithDiffColsAndTags() throws Exception { TableName tableName = TableName.valueOf(TEST_NAME.getMethodName()); - HTable table = null; + Table table = null; try { Admin hBaseAdmin = TEST_UTIL.getHBaseAdmin(); HColumnDescriptor colDesc = new HColumnDescriptor(fam); @@ -1496,7 +1497,7 @@ public class TestVisibilityLabelsWithDeletes { @Override public Void run() throws Exception { try { - HTable table = new HTable(conf, TEST_NAME.getMethodName()); + Table table = new HTable(conf, TEST_NAME.getMethodName()); Delete d = new Delete(row1); d.setCellVisibility(new CellVisibility(SECRET)); d.deleteColumns(fam, qual, 126l); @@ -1528,7 +1529,7 @@ public class TestVisibilityLabelsWithDeletes { @Test public void testDeleteColumnsWithDiffColsAndTags1() throws Exception { TableName tableName = TableName.valueOf(TEST_NAME.getMethodName()); - HTable table = null; + Table table = null; try { Admin hBaseAdmin = TEST_UTIL.getHBaseAdmin(); HColumnDescriptor colDesc = new HColumnDescriptor(fam); @@ -1550,7 +1551,7 @@ public class TestVisibilityLabelsWithDeletes { @Override public Void run() throws Exception { try { - HTable table = new HTable(conf, TEST_NAME.getMethodName()); + Table table = new HTable(conf, TEST_NAME.getMethodName()); Delete d = new Delete(row1); d.setCellVisibility(new CellVisibility(SECRET)); d.deleteColumns(fam, qual, 126l); @@ -1583,7 +1584,7 @@ public class TestVisibilityLabelsWithDeletes { public void testDeleteFamilyWithoutCellVisibilityWithMulipleVersions() throws Exception { setAuths(); TableName tableName = TableName.valueOf(TEST_NAME.getMethodName()); - HTable table = null; + Table table = null; try { table = doPutsWithoutVisibility(tableName); TEST_UTIL.getHBaseAdmin().flush(tableName); @@ -1591,7 +1592,7 @@ public class TestVisibilityLabelsWithDeletes { @Override public Void run() throws Exception { try { - HTable table = new HTable(conf, TEST_NAME.getMethodName()); + Table table = new HTable(conf, TEST_NAME.getMethodName()); Delete d = new Delete(row1); d.deleteFamily(fam); table.delete(d); @@ -1628,14 +1629,14 @@ public class TestVisibilityLabelsWithDeletes { throws Exception { setAuths(); TableName tableName = TableName.valueOf(TEST_NAME.getMethodName()); - HTable table = null; + Table table = null; try { table = doPutsWithoutVisibility(tableName); PrivilegedExceptionAction actiona = new PrivilegedExceptionAction() { @Override public Void run() throws Exception { try { - HTable table = new HTable(conf, TEST_NAME.getMethodName()); + Table table = new HTable(conf, TEST_NAME.getMethodName()); Delete d = new Delete(row1); d.setCellVisibility(new CellVisibility(SECRET + "&" + TOPSECRET)); d.deleteFamily(fam); @@ -1696,7 +1697,7 @@ public class TestVisibilityLabelsWithDeletes { public void testDeleteFamilySpecificTimeStampWithMulipleVersions() throws Exception { setAuths(); TableName tableName = TableName.valueOf(TEST_NAME.getMethodName()); - HTable table = null; + Table table = null; try { table = doPuts(tableName); TEST_UTIL.getHBaseAdmin().flush(tableName); @@ -1704,7 +1705,7 @@ public class TestVisibilityLabelsWithDeletes { @Override public Void run() throws Exception { try { - HTable table = new HTable(conf, TEST_NAME.getMethodName()); + Table table = new HTable(conf, TEST_NAME.getMethodName()); Delete d = new Delete(row1); d.setCellVisibility(new CellVisibility("(" + PRIVATE + "&" + CONFIDENTIAL + ")|(" + SECRET + "&" + TOPSECRET + ")")); @@ -1757,7 +1758,7 @@ public class TestVisibilityLabelsWithDeletes { public void testScanAfterCompaction() throws Exception { setAuths(); TableName tableName = TableName.valueOf(TEST_NAME.getMethodName()); - HTable table = null; + Table table = null; try { table = doPuts(tableName); TEST_UTIL.getHBaseAdmin().flush(tableName); @@ -1765,7 +1766,7 @@ public class TestVisibilityLabelsWithDeletes { @Override public Void run() throws Exception { try { - HTable table = new HTable(conf, TEST_NAME.getMethodName()); + Table table = new HTable(conf, TEST_NAME.getMethodName()); Delete d = new Delete(row1); d.setCellVisibility(new CellVisibility("(" + PRIVATE + "&" + CONFIDENTIAL + ")|(" + SECRET + "&" + TOPSECRET+")")); @@ -1816,7 +1817,7 @@ public class TestVisibilityLabelsWithDeletes { public void testDeleteFamilySpecificTimeStampWithMulipleVersionsDoneTwice() throws Exception { setAuths(); TableName tableName = TableName.valueOf(TEST_NAME.getMethodName()); - HTable table = null; + Table table = null; try { // Do not flush here. table = doPuts(tableName); @@ -1824,7 +1825,7 @@ public class TestVisibilityLabelsWithDeletes { @Override public Void run() throws Exception { try { - HTable table = new HTable(conf, TEST_NAME.getMethodName()); + Table table = new HTable(conf, TEST_NAME.getMethodName()); Delete d = new Delete(row1); d.setCellVisibility(new CellVisibility("(" + PRIVATE + "&" + CONFIDENTIAL + ")|(" + TOPSECRET + "&" + SECRET+")")); @@ -1876,7 +1877,7 @@ public class TestVisibilityLabelsWithDeletes { @Override public Void run() throws Exception { try { - HTable table = new HTable(conf, TEST_NAME.getMethodName()); + Table table = new HTable(conf, TEST_NAME.getMethodName()); Delete d = new Delete(row1); d.setCellVisibility(new CellVisibility("(" + CONFIDENTIAL + "&" + PRIVATE + ")|(" + TOPSECRET + "&" + SECRET+")")); @@ -1935,13 +1936,13 @@ public class TestVisibilityLabelsWithDeletes { }; VisibilityLabelsResponse response = SUPERUSER.runAs(action); TableName tableName = TableName.valueOf(TEST_NAME.getMethodName()); - HTable table = doPuts(tableName); + Table table = doPuts(tableName); try { PrivilegedExceptionAction actiona = new PrivilegedExceptionAction() { @Override public Void run() throws Exception { try { - HTable table = new HTable(conf, TEST_NAME.getMethodName()); + Table table = new HTable(conf, TEST_NAME.getMethodName()); Delete d = new Delete(row1); d.setCellVisibility(new CellVisibility(CONFIDENTIAL)); d.deleteFamilyVersion(fam, 123l); @@ -1992,13 +1993,13 @@ public class TestVisibilityLabelsWithDeletes { public void testSpecificDeletesFollowedByDeleteFamily() throws Exception { setAuths(); TableName tableName = TableName.valueOf(TEST_NAME.getMethodName()); - HTable table = doPuts(tableName); + Table table = doPuts(tableName); try { PrivilegedExceptionAction actiona = new PrivilegedExceptionAction() { @Override public Void run() throws Exception { try { - HTable table = new HTable(conf, TEST_NAME.getMethodName()); + Table table = new HTable(conf, TEST_NAME.getMethodName()); Delete d = new Delete(row1); d.setCellVisibility(new CellVisibility("(" + CONFIDENTIAL + "&" + PRIVATE + ")|(" + TOPSECRET + "&" + SECRET + ")")); @@ -2044,7 +2045,7 @@ public class TestVisibilityLabelsWithDeletes { @Override public Void run() throws Exception { try { - HTable table = new HTable(conf, TEST_NAME.getMethodName()); + Table table = new HTable(conf, TEST_NAME.getMethodName()); Delete d = new Delete(row1); d.setCellVisibility(new CellVisibility(CONFIDENTIAL)); d.deleteFamily(fam); @@ -2096,13 +2097,13 @@ public class TestVisibilityLabelsWithDeletes { }; VisibilityLabelsResponse response = SUPERUSER.runAs(action); TableName tableName = TableName.valueOf(TEST_NAME.getMethodName()); - HTable table = doPuts(tableName); + Table table = doPuts(tableName); try { PrivilegedExceptionAction actiona = new PrivilegedExceptionAction() { @Override public Void run() throws Exception { try { - HTable table = new HTable(conf, TEST_NAME.getMethodName()); + Table table = new HTable(conf, TEST_NAME.getMethodName()); Delete d = new Delete(row1); d.setCellVisibility(new CellVisibility("(" + CONFIDENTIAL + "&" + PRIVATE + ")|(" + TOPSECRET + "&" + SECRET + ")")); @@ -2149,7 +2150,7 @@ public class TestVisibilityLabelsWithDeletes { @Override public Void run() throws Exception { try { - HTable table = new HTable(conf, TEST_NAME.getMethodName()); + Table table = new HTable(conf, TEST_NAME.getMethodName()); Delete d = new Delete(row1); d.setCellVisibility(new CellVisibility(CONFIDENTIAL)); d.deleteFamily(fam); @@ -2190,7 +2191,7 @@ public class TestVisibilityLabelsWithDeletes { public void testDeleteColumnSpecificTimeStampWithMulipleVersionsDoneTwice() throws Exception { setAuths(); TableName tableName = TableName.valueOf(TEST_NAME.getMethodName()); - HTable table = null; + Table table = null; try { // Do not flush here. table = doPuts(tableName); @@ -2198,7 +2199,7 @@ public class TestVisibilityLabelsWithDeletes { @Override public Void run() throws Exception { try { - HTable table = new HTable(conf, TEST_NAME.getMethodName()); + Table table = new HTable(conf, TEST_NAME.getMethodName()); Delete d = new Delete(row1); d.setCellVisibility(new CellVisibility(SECRET + "&" + TOPSECRET)); d.deleteColumn(fam, qual, 125l); @@ -2249,7 +2250,7 @@ public class TestVisibilityLabelsWithDeletes { @Override public Void run() throws Exception { try { - HTable table = new HTable(conf, TEST_NAME.getMethodName()); + Table table = new HTable(conf, TEST_NAME.getMethodName()); Delete d = new Delete(row1); d.setCellVisibility(new CellVisibility("(" + CONFIDENTIAL + "&" + PRIVATE + ")|(" + TOPSECRET + "&" + SECRET+")")); @@ -2301,7 +2302,7 @@ public class TestVisibilityLabelsWithDeletes { public void testDeleteColumnSpecificTimeStampWithMulipleVersionsDoneTwice1() throws Exception { setAuths(); TableName tableName = TableName.valueOf(TEST_NAME.getMethodName()); - HTable table = null; + Table table = null; try { // Do not flush here. table = doPuts(tableName); @@ -2309,7 +2310,7 @@ public class TestVisibilityLabelsWithDeletes { @Override public Void run() throws Exception { try { - HTable table = new HTable(conf, TEST_NAME.getMethodName()); + Table table = new HTable(conf, TEST_NAME.getMethodName()); Delete d = new Delete(row1); d.setCellVisibility(new CellVisibility("(" + CONFIDENTIAL + "&" + PRIVATE + ")" + "|(" + TOPSECRET + "&" + SECRET + ")")); @@ -2361,7 +2362,7 @@ public class TestVisibilityLabelsWithDeletes { @Override public Void run() throws Exception { try { - HTable table = new HTable(conf, TEST_NAME.getMethodName()); + Table table = new HTable(conf, TEST_NAME.getMethodName()); Delete d = new Delete(row1); d.setCellVisibility(new CellVisibility(SECRET + "&" + TOPSECRET)); d.deleteColumn(fam, qual, 127l); @@ -2416,7 +2417,7 @@ public class TestVisibilityLabelsWithDeletes { public void testDeleteColumnSpecificTimeStampWithMulipleVersionsDoneTwice2() throws Exception { setAuths(); TableName tableName = TableName.valueOf(TEST_NAME.getMethodName()); - HTable table = null; + Table table = null; try { // Do not flush here. table = doPuts(tableName); @@ -2424,7 +2425,7 @@ public class TestVisibilityLabelsWithDeletes { @Override public Void run() throws Exception { try { - HTable table = new HTable(conf, TEST_NAME.getMethodName()); + Table table = new HTable(conf, TEST_NAME.getMethodName()); Delete d = new Delete(row1); d.setCellVisibility(new CellVisibility("(" + PRIVATE + "&" + CONFIDENTIAL + ")|(" + TOPSECRET + "&" + SECRET+")")); @@ -2481,7 +2482,7 @@ public class TestVisibilityLabelsWithDeletes { @Override public Void run() throws Exception { try { - HTable table = new HTable(conf, TEST_NAME.getMethodName()); + Table table = new HTable(conf, TEST_NAME.getMethodName()); Delete d = new Delete(row1); d.setCellVisibility(new CellVisibility("(" + CONFIDENTIAL + "&" + PRIVATE + ")|(" + TOPSECRET + "&" + SECRET+")")); @@ -2538,7 +2539,7 @@ public class TestVisibilityLabelsWithDeletes { throws Exception { setAuths(); TableName tableName = TableName.valueOf(TEST_NAME.getMethodName()); - HTable table = null; + Table table = null; try { // Do not flush here. table = doPuts(tableName); @@ -2546,7 +2547,7 @@ public class TestVisibilityLabelsWithDeletes { @Override public Void run() throws Exception { try { - HTable table = new HTable(conf, TEST_NAME.getMethodName()); + Table table = new HTable(conf, TEST_NAME.getMethodName()); Delete d = new Delete(row1); d.setCellVisibility(new CellVisibility(SECRET + "&" + TOPSECRET)); d.deleteColumn(fam, qual, 125l); @@ -2597,7 +2598,7 @@ public class TestVisibilityLabelsWithDeletes { @Override public Void run() throws Exception { try { - HTable table = new HTable(conf, TEST_NAME.getMethodName()); + Table table = new HTable(conf, TEST_NAME.getMethodName()); Delete d = new Delete(row1); d.setCellVisibility(new CellVisibility("(" + CONFIDENTIAL + "&" + PRIVATE + ")|(" + TOPSECRET + "&" + SECRET+")")); @@ -2660,7 +2661,7 @@ public class TestVisibilityLabelsWithDeletes { public void testDiffDeleteTypesForTheSameCellUsingMultipleVersions() throws Exception { setAuths(); TableName tableName = TableName.valueOf(TEST_NAME.getMethodName()); - HTable table = null; + Table table = null; try { // Do not flush here. table = doPuts(tableName); @@ -2668,7 +2669,7 @@ public class TestVisibilityLabelsWithDeletes { @Override public Void run() throws Exception { try { - HTable table = new HTable(conf, TEST_NAME.getMethodName()); + Table table = new HTable(conf, TEST_NAME.getMethodName()); Delete d = new Delete(row1); d.setCellVisibility(new CellVisibility("(" + PRIVATE + "&" + CONFIDENTIAL + ")|(" + TOPSECRET + "&" + SECRET+")")); @@ -2720,7 +2721,7 @@ public class TestVisibilityLabelsWithDeletes { @Override public Void run() throws Exception { try { - HTable table = new HTable(conf, TEST_NAME.getMethodName()); + Table table = new HTable(conf, TEST_NAME.getMethodName()); Delete d = new Delete(row1); d.setCellVisibility(new CellVisibility("(" + CONFIDENTIAL + "&" + PRIVATE + ")|(" + TOPSECRET + "&" + SECRET+")")); @@ -2771,7 +2772,7 @@ public class TestVisibilityLabelsWithDeletes { public void testDeleteColumnLatestWithNoCellVisibility() throws Exception { setAuths(); TableName tableName = TableName.valueOf(TEST_NAME.getMethodName()); - HTable table = null; + Table table = null; try { table = doPuts(tableName); TEST_UTIL.getHBaseAdmin().flush(tableName); @@ -2779,7 +2780,7 @@ public class TestVisibilityLabelsWithDeletes { @Override public Void run() throws Exception { try { - HTable table = new HTable(conf, TEST_NAME.getMethodName()); + Table table = new HTable(conf, TEST_NAME.getMethodName()); Delete d = new Delete(row1); d.deleteColumn(fam, qual, 125l); table.delete(d); @@ -2803,7 +2804,7 @@ public class TestVisibilityLabelsWithDeletes { @Override public Void run() throws Exception { try { - HTable table = new HTable(conf, TEST_NAME.getMethodName()); + Table table = new HTable(conf, TEST_NAME.getMethodName()); Delete d = new Delete(row1); d.deleteColumns(fam, qual, 125l); table.delete(d); @@ -2828,7 +2829,7 @@ public class TestVisibilityLabelsWithDeletes { @Override public Void run() throws Exception { try { - HTable table = new HTable(conf, TEST_NAME.getMethodName()); + Table table = new HTable(conf, TEST_NAME.getMethodName()); Delete d = new Delete(row1); d.deleteFamily(fam, 125l); table.delete(d); @@ -2853,7 +2854,7 @@ public class TestVisibilityLabelsWithDeletes { @Override public Void run() throws Exception { try { - HTable table = new HTable(conf, TEST_NAME.getMethodName()); + Table table = new HTable(conf, TEST_NAME.getMethodName()); Delete d = new Delete(row1); d.deleteFamily(fam); table.delete(d); @@ -2878,7 +2879,7 @@ public class TestVisibilityLabelsWithDeletes { @Override public Void run() throws Exception { try { - HTable table = new HTable(conf, TEST_NAME.getMethodName()); + Table table = new HTable(conf, TEST_NAME.getMethodName()); Delete d = new Delete(row1); d.deleteColumns(fam, qual); table.delete(d); @@ -2903,7 +2904,7 @@ public class TestVisibilityLabelsWithDeletes { @Override public Void run() throws Exception { try { - HTable table = new HTable(conf, TEST_NAME.getMethodName()); + Table table = new HTable(conf, TEST_NAME.getMethodName()); Delete d = new Delete(row1); d.deleteFamilyVersion(fam, 126l); table.delete(d); @@ -2968,7 +2969,7 @@ public class TestVisibilityLabelsWithDeletes { public void testVisibilityExpressionWithNotEqualORCondition() throws Exception { setAuths(); TableName tableName = TableName.valueOf(TEST_NAME.getMethodName()); - HTable table = null; + Table table = null; try { Admin hBaseAdmin = TEST_UTIL.getHBaseAdmin(); HColumnDescriptor colDesc = new HColumnDescriptor(fam); @@ -2990,7 +2991,7 @@ public class TestVisibilityLabelsWithDeletes { @Override public Void run() throws Exception { try { - HTable table = new HTable(conf, TEST_NAME.getMethodName()); + Table table = new HTable(conf, TEST_NAME.getMethodName()); Delete d = new Delete(row1); d.deleteColumn(fam, qual, 124l); d.setCellVisibility(new CellVisibility(PRIVATE )); @@ -3028,9 +3029,9 @@ public class TestVisibilityLabelsWithDeletes { } } - public static HTable createTableAndWriteDataWithLabels(TableName tableName, String... labelExps) + public static Table createTableAndWriteDataWithLabels(TableName tableName, String... labelExps) throws Exception { - HTable table = null; + Table table = null; table = TEST_UTIL.createTable(tableName, fam); int i = 1; List puts = new ArrayList(); @@ -3046,9 +3047,9 @@ public class TestVisibilityLabelsWithDeletes { return table; } - public static HTable createTableAndWriteDataWithLabels(TableName tableName, long[] timestamp, + public static Table createTableAndWriteDataWithLabels(TableName tableName, long[] timestamp, String... labelExps) throws Exception { - HTable table = null; + Table table = null; table = TEST_UTIL.createTable(tableName, fam); int i = 1; List puts = new ArrayList(); diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/security/visibility/TestVisibilityLabelsWithSLGStack.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/security/visibility/TestVisibilityLabelsWithSLGStack.java index 6cf1ebab8ba..2815d62e414 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/security/visibility/TestVisibilityLabelsWithSLGStack.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/security/visibility/TestVisibilityLabelsWithSLGStack.java @@ -29,11 +29,11 @@ import org.apache.hadoop.hbase.HBaseTestingUtility; import org.apache.hadoop.hbase.HConstants; import org.apache.hadoop.hbase.MediumTests; import org.apache.hadoop.hbase.TableName; -import org.apache.hadoop.hbase.client.HTable; import org.apache.hadoop.hbase.client.Put; import org.apache.hadoop.hbase.client.Result; import org.apache.hadoop.hbase.client.ResultScanner; import org.apache.hadoop.hbase.client.Scan; +import org.apache.hadoop.hbase.client.Table; import org.apache.hadoop.hbase.protobuf.generated.VisibilityLabelsProtos.VisibilityLabelsResponse; import org.apache.hadoop.hbase.security.User; import org.apache.hadoop.hbase.util.Bytes; @@ -81,7 +81,7 @@ public class TestVisibilityLabelsWithSLGStack { @Test public void testWithSAGStack() throws Exception { TableName tableName = TableName.valueOf(TEST_NAME.getMethodName()); - HTable table = null; + Table table = null; try { table = TEST_UTIL.createTable(tableName, CF); Put put = new Put(ROW_1); diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/security/visibility/TestVisibilityWithCheckAuths.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/security/visibility/TestVisibilityWithCheckAuths.java index 52a4960bd37..ff0d29f6838 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/security/visibility/TestVisibilityWithCheckAuths.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/security/visibility/TestVisibilityWithCheckAuths.java @@ -34,6 +34,7 @@ import org.apache.hadoop.hbase.client.Admin; import org.apache.hadoop.hbase.client.Append; import org.apache.hadoop.hbase.client.HTable; import org.apache.hadoop.hbase.client.Put; +import org.apache.hadoop.hbase.client.Table; import org.apache.hadoop.hbase.protobuf.generated.VisibilityLabelsProtos.VisibilityLabelsResponse; import org.apache.hadoop.hbase.security.User; import org.apache.hadoop.hbase.util.Bytes; @@ -125,13 +126,13 @@ public class TestVisibilityWithCheckAuths { HTableDescriptor desc = new HTableDescriptor(tableName); desc.addFamily(colDesc); hBaseAdmin.createTable(desc); - HTable table = null; + Table table = null; try { TEST_UTIL.getHBaseAdmin().flush(tableName); PrivilegedExceptionAction actiona = new PrivilegedExceptionAction() { @Override public Void run() throws Exception { - HTable table = null; + Table table = null; try { table = new HTable(conf, TEST_NAME.getMethodName()); Put p = new Put(row1); @@ -169,7 +170,7 @@ public class TestVisibilityWithCheckAuths { }; SUPERUSER.runAs(action); TableName tableName = TableName.valueOf(TEST_NAME.getMethodName()); - HTable table = null; + Table table = null; try { table = TEST_UTIL.createTable(tableName, fam); final byte[] row1 = Bytes.toBytes("row1"); @@ -177,7 +178,7 @@ public class TestVisibilityWithCheckAuths { PrivilegedExceptionAction actiona = new PrivilegedExceptionAction() { @Override public Void run() throws Exception { - HTable table = null; + Table table = null; try { table = new HTable(conf, TEST_NAME.getMethodName()); Put put = new Put(row1); @@ -194,7 +195,7 @@ public class TestVisibilityWithCheckAuths { actiona = new PrivilegedExceptionAction() { @Override public Void run() throws Exception { - HTable table = null; + Table table = null; try { table = new HTable(conf, TEST_NAME.getMethodName()); Append append = new Append(row1); @@ -210,7 +211,7 @@ public class TestVisibilityWithCheckAuths { actiona = new PrivilegedExceptionAction() { @Override public Void run() throws Exception { - HTable table = null; + Table table = null; try { table = new HTable(conf, TEST_NAME.getMethodName()); Append append = new Append(row1); diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/snapshot/SnapshotTestingUtils.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/snapshot/SnapshotTestingUtils.java index 35b961c8a4c..c147fd07088 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/snapshot/SnapshotTestingUtils.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/snapshot/SnapshotTestingUtils.java @@ -46,9 +46,9 @@ import org.apache.hadoop.hbase.TableName; import org.apache.hadoop.hbase.TableNotEnabledException; import org.apache.hadoop.hbase.client.Admin; import org.apache.hadoop.hbase.client.Durability; -import org.apache.hadoop.hbase.client.HBaseAdmin; import org.apache.hadoop.hbase.client.HTable; import org.apache.hadoop.hbase.client.Put; +import org.apache.hadoop.hbase.client.Table; import org.apache.hadoop.hbase.errorhandling.ForeignExceptionDispatcher; import org.apache.hadoop.hbase.client.RegionReplicaUtil; import org.apache.hadoop.hbase.io.HFileLink; @@ -692,7 +692,7 @@ public class SnapshotTestingUtils { waitForTableToBeOnline(util, table.getName()); } - private static void putData(final HTable table, final byte[][] families, + private static void putData(final Table table, final byte[][] families, final byte[] key, final byte[] value) throws IOException { byte[] q = Bytes.toBytes("q"); Put put = new Put(key); @@ -722,7 +722,7 @@ public class SnapshotTestingUtils { public static void verifyRowCount(final HBaseTestingUtility util, final TableName tableName, long expectedRows) throws IOException { - HTable table = new HTable(util.getConfiguration(), tableName); + Table table = new HTable(util.getConfiguration(), tableName); try { assertEquals(expectedRows, util.countRows(table)); } finally { diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/snapshot/TestExportSnapshot.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/snapshot/TestExportSnapshot.java index e4e2719e7e5..d0d1785490b 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/snapshot/TestExportSnapshot.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/snapshot/TestExportSnapshot.java @@ -31,30 +31,24 @@ import java.util.Set; import org.apache.commons.logging.Log; import org.apache.commons.logging.LogFactory; import org.apache.hadoop.conf.Configuration; -import org.apache.hadoop.fs.FSDataOutputStream; import org.apache.hadoop.fs.FileStatus; import org.apache.hadoop.fs.FileSystem; -import org.apache.hadoop.fs.FileUtil; import org.apache.hadoop.fs.Path; import org.apache.hadoop.hbase.HBaseTestingUtility; -import org.apache.hadoop.hbase.HColumnDescriptor; import org.apache.hadoop.hbase.HConstants; import org.apache.hadoop.hbase.HRegionInfo; -import org.apache.hadoop.hbase.HTableDescriptor; import org.apache.hadoop.hbase.MediumTests; import org.apache.hadoop.hbase.TableName; import org.apache.hadoop.hbase.client.Admin; -import org.apache.hadoop.hbase.client.HBaseAdmin; import org.apache.hadoop.hbase.client.HTable; +import org.apache.hadoop.hbase.client.Table; import org.apache.hadoop.hbase.master.snapshot.SnapshotManager; import org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.SnapshotDescription; import org.apache.hadoop.hbase.protobuf.generated.SnapshotProtos.SnapshotFileInfo; import org.apache.hadoop.hbase.protobuf.generated.SnapshotProtos.SnapshotRegionManifest; import org.apache.hadoop.hbase.snapshot.SnapshotTestingUtils.SnapshotMock; -import org.apache.hadoop.hbase.regionserver.HRegionFileSystem; import org.apache.hadoop.hbase.util.Bytes; import org.apache.hadoop.hbase.util.FSUtils; -import org.apache.hadoop.hbase.util.FSTableDescriptors; import org.apache.hadoop.hbase.util.Pair; import org.junit.After; import org.junit.AfterClass; @@ -121,7 +115,7 @@ public class TestExportSnapshot { admin.snapshot(emptySnapshotName, tableName); // Add some rows - HTable table = new HTable(TEST_UTIL.getConfiguration(), tableName); + Table table = new HTable(TEST_UTIL.getConfiguration(), tableName); SnapshotTestingUtils.loadData(TEST_UTIL, tableName, 50, FAMILY); tableNumFiles = admin.getTableRegions(tableName).size(); diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/trace/TestHTraceHooks.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/trace/TestHTraceHooks.java index 859bff2303a..0925fe375f5 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/trace/TestHTraceHooks.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/trace/TestHTraceHooks.java @@ -26,8 +26,8 @@ import java.util.Collection; import org.apache.hadoop.hbase.HBaseTestingUtility; import org.apache.hadoop.hbase.MediumTests; import org.apache.hadoop.hbase.Waiter; -import org.apache.hadoop.hbase.client.HTable; import org.apache.hadoop.hbase.client.Put; +import org.apache.hadoop.hbase.client.Table; import org.htrace.Sampler; import org.htrace.Span; import org.htrace.Trace; @@ -63,7 +63,7 @@ public class TestHTraceHooks { @Test public void testTraceCreateTable() throws Exception { TraceScope tableCreationSpan = Trace.startSpan("creating table", Sampler.ALWAYS); - HTable table; + Table table; try { table = TEST_UTIL.createTable("table".getBytes(), diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/util/LoadTestTool.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/util/LoadTestTool.java index 9e7186a597b..57108878a56 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/util/LoadTestTool.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/util/LoadTestTool.java @@ -40,6 +40,7 @@ import org.apache.hadoop.hbase.HColumnDescriptor; import org.apache.hadoop.hbase.HConstants; import org.apache.hadoop.hbase.HTableDescriptor; import org.apache.hadoop.hbase.TableName; +import org.apache.hadoop.hbase.client.Admin; import org.apache.hadoop.hbase.client.Durability; import org.apache.hadoop.hbase.client.HBaseAdmin; import org.apache.hadoop.hbase.io.compress.Compression; @@ -247,7 +248,7 @@ public class LoadTestTool extends AbstractHBaseTool { */ protected void applyColumnFamilyOptions(TableName tableName, byte[][] columnFamilies) throws IOException { - HBaseAdmin admin = new HBaseAdmin(conf); + Admin admin = new HBaseAdmin(conf); HTableDescriptor tableDesc = admin.getTableDescriptor(tableName); LOG.info("Disabling table " + tableName); admin.disableTable(tableName); diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/util/MultiThreadedReader.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/util/MultiThreadedReader.java index cce71307e6f..c47c3287d86 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/util/MultiThreadedReader.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/util/MultiThreadedReader.java @@ -33,6 +33,7 @@ import org.apache.hadoop.hbase.client.Get; import org.apache.hadoop.hbase.client.Consistency; import org.apache.hadoop.hbase.client.HTableInterface; import org.apache.hadoop.hbase.client.Result; +import org.apache.hadoop.hbase.client.Table; import org.apache.hadoop.hbase.util.test.LoadTestDataGenerator; /** Creates multiple threads that read and verify previously written data */ @@ -134,7 +135,7 @@ public class MultiThreadedReader extends MultiThreadedAction public class HBaseReaderThread extends Thread { protected final int readerId; - protected final HTableInterface table; + protected final Table table; /** The "current" key being read. Increases from startKey to endKey. */ private long curKey; @@ -353,7 +354,7 @@ public class MultiThreadedReader extends MultiThreadedAction } protected void verifyResultsAndUpdateMetrics(boolean verify, Get[] gets, long elapsedNano, - Result[] results, HTableInterface table, boolean isNullExpected) + Result[] results, Table table, boolean isNullExpected) throws IOException { totalOpTimeMs.addAndGet(elapsedNano / 1000000); numKeys.addAndGet(gets.length); @@ -365,14 +366,14 @@ public class MultiThreadedReader extends MultiThreadedAction } protected void verifyResultsAndUpdateMetrics(boolean verify, Get get, long elapsedNano, - Result result, HTableInterface table, boolean isNullExpected) + Result result, Table table, boolean isNullExpected) throws IOException { verifyResultsAndUpdateMetrics(verify, new Get[]{get}, elapsedNano, new Result[]{result}, table, isNullExpected); } private void verifyResultsAndUpdateMetricsOnAPerGetBasis(boolean verify, Get get, - Result result, HTableInterface table, boolean isNullExpected) throws IOException { + Result result, Table table, boolean isNullExpected) throws IOException { if (!result.isEmpty()) { if (verify) { numKeysVerified.incrementAndGet(); diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/util/MultiThreadedReaderWithACL.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/util/MultiThreadedReaderWithACL.java index 5c70908e1cb..8635c11c0c6 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/util/MultiThreadedReaderWithACL.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/util/MultiThreadedReaderWithACL.java @@ -29,6 +29,7 @@ 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.Result; +import org.apache.hadoop.hbase.client.Table; import org.apache.hadoop.hbase.security.User; import org.apache.hadoop.hbase.util.test.LoadTestDataGenerator; import org.apache.hadoop.security.UserGroupInformation; @@ -44,7 +45,7 @@ public class MultiThreadedReaderWithACL extends MultiThreadedReader { * Maps user with Table instance. Because the table instance has to be created * per user inorder to work in that user's context */ - private Map userVsTable = new HashMap(); + private Map userVsTable = new HashMap(); private Map users = new HashMap(); private String[] userNames; @@ -75,7 +76,7 @@ public class MultiThreadedReaderWithACL extends MultiThreadedReader { @Override protected void closeTable() { - for (HTableInterface table : userVsTable.values()) { + for (Table table : userVsTable.values()) { try { table.close(); } catch (Exception e) { @@ -94,7 +95,7 @@ public class MultiThreadedReaderWithACL extends MultiThreadedReader { PrivilegedExceptionAction action = new PrivilegedExceptionAction() { @Override public Object run() throws Exception { - HTableInterface localTable = null; + Table localTable = null; try { Result result = null; int specialPermCellInsertionFactor = Integer.parseInt(dataGenerator.getArgs()[2]); diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/util/MultiThreadedUpdater.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/util/MultiThreadedUpdater.java index 5d031fd9ad6..464f44b1c81 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/util/MultiThreadedUpdater.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/util/MultiThreadedUpdater.java @@ -46,6 +46,7 @@ import org.apache.hadoop.hbase.client.Mutation; import org.apache.hadoop.hbase.client.Put; import org.apache.hadoop.hbase.client.Result; import org.apache.hadoop.hbase.client.RetriesExhaustedWithDetailsException; +import org.apache.hadoop.hbase.client.Table; import org.apache.hadoop.hbase.exceptions.OperationConflictException; import org.apache.hadoop.hbase.protobuf.generated.ClientProtos.MutationProto.MutationType; import org.apache.hadoop.hbase.util.test.LoadTestDataGenerator; @@ -122,7 +123,7 @@ public class MultiThreadedUpdater extends MultiThreadedWriterBase { } protected class HBaseUpdaterThread extends Thread { - protected final HTableInterface table; + protected final Table table; public HBaseUpdaterThread(int updaterId) throws IOException { setName(getClass().getSimpleName() + "_" + updaterId); @@ -274,11 +275,11 @@ public class MultiThreadedUpdater extends MultiThreadedWriterBase { return result; } - public void mutate(HTableInterface table, Mutation m, long keyBase) { + public void mutate(Table table, Mutation m, long keyBase) { mutate(table, m, keyBase, null, null, null, null); } - public void mutate(HTableInterface table, Mutation m, + public void mutate(Table table, Mutation m, long keyBase, byte[] row, byte[] cf, byte[] q, byte[] v) { long start = System.currentTimeMillis(); try { @@ -327,11 +328,11 @@ public class MultiThreadedUpdater extends MultiThreadedWriterBase { } } - public void mutate(HTableInterface table, Mutation m, long keyBase) { + public void mutate(Table table, Mutation m, long keyBase) { mutate(table, m, keyBase, null, null, null, null); } - public void mutate(HTableInterface table, Mutation m, + public void mutate(Table table, Mutation m, long keyBase, byte[] row, byte[] cf, byte[] q, byte[] v) { long start = System.currentTimeMillis(); try { diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/util/MultiThreadedUpdaterWithACL.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/util/MultiThreadedUpdaterWithACL.java index 7c6f716e927..30d425c5a4e 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/util/MultiThreadedUpdaterWithACL.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/util/MultiThreadedUpdaterWithACL.java @@ -38,6 +38,7 @@ import org.apache.hadoop.hbase.client.Mutation; import org.apache.hadoop.hbase.client.Put; import org.apache.hadoop.hbase.client.Result; import org.apache.hadoop.hbase.client.RetriesExhaustedWithDetailsException; +import org.apache.hadoop.hbase.client.Table; import org.apache.hadoop.hbase.security.User; import org.apache.hadoop.hbase.util.test.LoadTestDataGenerator; import org.apache.hadoop.security.UserGroupInformation; @@ -54,7 +55,7 @@ public class MultiThreadedUpdaterWithACL extends MultiThreadedUpdater { * Maps user with Table instance. Because the table instance has to be created * per user inorder to work in that user's context */ - private Map userVsTable = new HashMap(); + private Map userVsTable = new HashMap(); private Map users = new HashMap(); private String[] userNames; @@ -76,7 +77,7 @@ public class MultiThreadedUpdaterWithACL extends MultiThreadedUpdater { public class HBaseUpdaterThreadWithACL extends HBaseUpdaterThread { - private HTableInterface table; + private Table table; private MutateAccessAction mutateAction = new MutateAccessAction(); public HBaseUpdaterThreadWithACL(int updaterId) throws IOException { @@ -94,7 +95,7 @@ public class MultiThreadedUpdaterWithACL extends MultiThreadedUpdater { if (table != null) { table.close(); } - for (HTableInterface table : userVsTable.values()) { + for (Table table : userVsTable.values()) { try { table.close(); } catch (Exception e) { @@ -113,7 +114,7 @@ public class MultiThreadedUpdaterWithACL extends MultiThreadedUpdater { @Override public Object run() throws Exception { Result res = null; - HTableInterface localTable = null; + Table localTable = null; try { int mod = ((int) rowKeyBase % userNames.length); if (userVsTable.get(userNames[mod]) == null) { @@ -160,7 +161,7 @@ public class MultiThreadedUpdaterWithACL extends MultiThreadedUpdater { } @Override - public void mutate(final HTableInterface table, Mutation m, final long keyBase, final byte[] row, + public void mutate(final Table table, Mutation m, final long keyBase, final byte[] row, final byte[] cf, final byte[] q, final byte[] v) { final long start = System.currentTimeMillis(); try { @@ -181,7 +182,7 @@ public class MultiThreadedUpdaterWithACL extends MultiThreadedUpdater { } class MutateAccessAction implements PrivilegedExceptionAction { - private HTableInterface table; + private Table table; private long start; private Mutation m; private long keyBase; diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/util/MultiThreadedWriter.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/util/MultiThreadedWriter.java index 618dabeabe9..4d1c286cf09 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/util/MultiThreadedWriter.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/util/MultiThreadedWriter.java @@ -36,6 +36,7 @@ import org.apache.hadoop.hbase.TableName; import org.apache.hadoop.hbase.client.HTableInterface; import org.apache.hadoop.hbase.client.Put; import org.apache.hadoop.hbase.client.RetriesExhaustedWithDetailsException; +import org.apache.hadoop.hbase.client.Table; import org.apache.hadoop.hbase.util.test.LoadTestDataGenerator; import org.apache.hadoop.util.StringUtils; @@ -79,7 +80,7 @@ public class MultiThreadedWriter extends MultiThreadedWriterBase { } public class HBaseWriterThread extends Thread { - private final HTableInterface table; + private final Table table; public HBaseWriterThread(int writerId) throws IOException { setName(getClass().getSimpleName() + "_" + writerId); @@ -138,7 +139,7 @@ public class MultiThreadedWriter extends MultiThreadedWriterBase { } } - public void insert(HTableInterface table, Put put, long keyBase) { + public void insert(Table table, Put put, long keyBase) { long start = System.currentTimeMillis(); try { put = (Put) dataGenerator.beforeMutate(keyBase, put); diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/util/MultiThreadedWriterBase.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/util/MultiThreadedWriterBase.java index f3455c01b73..9eb0c930765 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/util/MultiThreadedWriterBase.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/util/MultiThreadedWriterBase.java @@ -33,7 +33,7 @@ import org.apache.commons.logging.LogFactory; import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.hbase.HRegionLocation; import org.apache.hadoop.hbase.TableName; -import org.apache.hadoop.hbase.client.HTableInterface; +import org.apache.hadoop.hbase.client.Table; import org.apache.hadoop.hbase.util.test.LoadTestDataGenerator; /** Creates multiple threads that write key/values into the */ @@ -89,7 +89,7 @@ public abstract class MultiThreadedWriterBase extends MultiThreadedAction { } } - protected String getRegionDebugInfoSafe(HTableInterface table, byte[] rowKey) { + protected String getRegionDebugInfoSafe(Table table, byte[] rowKey) { HRegionLocation cached = null, real = null; try { cached = connection.getRegionLocation(tableName, rowKey, false); diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/util/MultiThreadedWriterWithACL.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/util/MultiThreadedWriterWithACL.java index ba452c207e3..14765679fcd 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/util/MultiThreadedWriterWithACL.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/util/MultiThreadedWriterWithACL.java @@ -30,6 +30,7 @@ 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.RetriesExhaustedWithDetailsException; +import org.apache.hadoop.hbase.client.Table; import org.apache.hadoop.hbase.security.User; import org.apache.hadoop.hbase.util.test.LoadTestDataGenerator; import org.apache.hadoop.util.StringUtils; @@ -63,7 +64,7 @@ public class MultiThreadedWriterWithACL extends MultiThreadedWriter { public class HBaseWriterThreadWithACL extends HBaseWriterThread { - private HTableInterface table; + private Table table; private WriteAccessAction writerAction = new WriteAccessAction(); public HBaseWriterThreadWithACL(int writerId) throws IOException { @@ -87,7 +88,7 @@ public class MultiThreadedWriterWithACL extends MultiThreadedWriter { } @Override - public void insert(final HTableInterface table, Put put, final long keyBase) { + public void insert(final Table table, Put put, final long keyBase) { final long start = System.currentTimeMillis(); try { put = (Put) dataGenerator.beforeMutate(keyBase, put); @@ -137,7 +138,7 @@ public class MultiThreadedWriterWithACL extends MultiThreadedWriter { } } - private void recordFailure(final HTableInterface table, final Put put, final long keyBase, + private void recordFailure(final Table table, final Put put, final long keyBase, final long start, IOException e) { failedKeySet.add(keyBase); String exceptionInfo; diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/util/RestartMetaTest.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/util/RestartMetaTest.java index 51d2c4a33c6..81b219d6198 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/util/RestartMetaTest.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/util/RestartMetaTest.java @@ -28,6 +28,7 @@ import org.apache.hadoop.hbase.client.HTable; import org.apache.hadoop.hbase.client.Result; import org.apache.hadoop.hbase.client.ResultScanner; import org.apache.hadoop.hbase.client.Scan; +import org.apache.hadoop.hbase.client.Table; import org.apache.hadoop.hbase.io.compress.Compression; import org.apache.hadoop.hbase.io.encoding.DataBlockEncoding; import org.apache.hadoop.hbase.util.test.LoadTestDataGenerator; @@ -120,7 +121,7 @@ public class RestartMetaTest extends AbstractHBaseTool { LOG.debug("Trying to scan meta"); - HTable metaTable = new HTable(conf, TableName.META_TABLE_NAME); + Table metaTable = new HTable(conf, TableName.META_TABLE_NAME); ResultScanner scanner = metaTable.getScanner(new Scan()); Result result; while ((result = scanner.next()) != null) { diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/util/TestCoprocessorScanPolicy.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/util/TestCoprocessorScanPolicy.java index dba1bba1861..4a092c80e93 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/util/TestCoprocessorScanPolicy.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/util/TestCoprocessorScanPolicy.java @@ -48,6 +48,7 @@ import org.apache.hadoop.hbase.client.Put; import org.apache.hadoop.hbase.client.Result; import org.apache.hadoop.hbase.client.Scan; import org.apache.hadoop.hbase.client.Durability; +import org.apache.hadoop.hbase.client.Table; import org.apache.hadoop.hbase.coprocessor.BaseRegionObserver; import org.apache.hadoop.hbase.coprocessor.CoprocessorHost; import org.apache.hadoop.hbase.coprocessor.ObserverContext; @@ -108,7 +109,7 @@ public class TestCoprocessorScanPolicy { if (TEST_UTIL.getHBaseAdmin().tableExists(tableName)) { TEST_UTIL.deleteTable(tableName); } - HTable t = TEST_UTIL.createTable(tableName, F, 1); + Table t = TEST_UTIL.createTable(tableName, F, 1); // set the version override to 2 Put p = new Put(R); p.setAttribute("versions", new byte[]{}); @@ -164,7 +165,7 @@ public class TestCoprocessorScanPolicy { .setTimeToLive(1); desc.addFamily(hcd); TEST_UTIL.getHBaseAdmin().createTable(desc); - HTable t = new HTable(new Configuration(TEST_UTIL.getConfiguration()), tableName); + Table t = new HTable(new Configuration(TEST_UTIL.getConfiguration()), tableName); long now = EnvironmentEdgeManager.currentTime(); ManualEnvironmentEdge me = new ManualEnvironmentEdge(); me.setValue(now); diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/util/TestHBaseFsck.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/util/TestHBaseFsck.java index 4c024400741..88d2d3558a9 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/util/TestHBaseFsck.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/util/TestHBaseFsck.java @@ -75,6 +75,7 @@ import org.apache.hadoop.hbase.client.Put; import org.apache.hadoop.hbase.client.Result; import org.apache.hadoop.hbase.client.ResultScanner; import org.apache.hadoop.hbase.client.Scan; +import org.apache.hadoop.hbase.client.Table; import org.apache.hadoop.hbase.io.hfile.TestHFile; import org.apache.hadoop.hbase.master.AssignmentManager; import org.apache.hadoop.hbase.master.HMaster; @@ -159,7 +160,7 @@ public class TestHBaseFsck { // Now let's mess it up and change the assignment in hbase:meta to // point to a different region server - HTable meta = new HTable(conf, HTableDescriptor.META_TABLEDESC.getTableName(), + Table meta = new HTable(conf, HTableDescriptor.META_TABLEDESC.getTableName(), executorService); Scan scan = new Scan(); scan.setStartRow(Bytes.toBytes(table+",,")); @@ -204,7 +205,7 @@ public class TestHBaseFsck { assertNoErrors(doFsck(conf, false)); // comment needed - what is the purpose of this line - HTable t = new HTable(conf, Bytes.toBytes(table), executorService); + Table t = new HTable(conf, Bytes.toBytes(table), executorService); ResultScanner s = t.getScanner(new Scan()); s.close(); t.close(); @@ -216,7 +217,7 @@ public class TestHBaseFsck { @Test(timeout=180000) public void testFixAssignmentsWhenMETAinTransition() throws Exception { MiniHBaseCluster cluster = TEST_UTIL.getHBaseCluster(); - HBaseAdmin admin = null; + Admin admin = null; try { admin = new HBaseAdmin(TEST_UTIL.getConfiguration()); admin.closeRegion(cluster.getServerHoldingMeta(), @@ -241,7 +242,7 @@ public class TestHBaseFsck { private HRegionInfo createRegion(Configuration conf, final HTableDescriptor htd, byte[] startKey, byte[] endKey) throws IOException { - HTable meta = new HTable(conf, TableName.META_TABLE_NAME, executorService); + Table meta = new HTable(conf, TableName.META_TABLE_NAME, executorService); HRegionInfo hri = new HRegionInfo(htd.getTableName(), startKey, endKey); MetaTableAccessor.addRegionToMeta(meta, hri); meta.close(); @@ -337,7 +338,7 @@ public class TestHBaseFsck { } if (metaRow) { - HTable meta = new HTable(conf, TableName.META_TABLE_NAME, executorService); + Table meta = new HTable(conf, TableName.META_TABLE_NAME, executorService); Delete delete = new Delete(deleteRow); meta.delete(delete); } @@ -357,7 +358,7 @@ public class TestHBaseFsck { * @throws InterruptedException * @throws KeeperException */ - HTable setupTable(TableName tablename) throws Exception { + Table setupTable(TableName tablename) throws Exception { return setupTableWithRegionReplica(tablename, 1); } @@ -368,7 +369,7 @@ public class TestHBaseFsck { * @return * @throws Exception */ - HTable setupTableWithRegionReplica(TableName tablename, int replicaCount) throws Exception { + Table setupTableWithRegionReplica(TableName tablename, int replicaCount) throws Exception { HTableDescriptor desc = new HTableDescriptor(tablename); desc.setRegionReplication(replicaCount); HColumnDescriptor hcd = new HColumnDescriptor(Bytes.toString(FAM)); @@ -407,7 +408,7 @@ public class TestHBaseFsck { * @throws IOException */ void deleteTable(TableName tablename) throws IOException { - HBaseAdmin admin = new HBaseAdmin(conf); + Admin admin = new HBaseAdmin(conf); admin.getConnection().clearRegionCache(); if (admin.isTableEnabled(tablename)) { admin.disableTableAsync(tablename); @@ -813,7 +814,7 @@ public class TestHBaseFsck { assertNotNull(regionName); assertNotNull(serverName); - HTable meta = new HTable(conf, TableName.META_TABLE_NAME, executorService); + Table meta = new HTable(conf, TableName.META_TABLE_NAME, executorService); Put put = new Put(regionName); put.add(HConstants.CATALOG_FAMILY, HConstants.SERVER_QUALIFIER, Bytes.toBytes(serverName.getHostAndPort())); @@ -1295,7 +1296,7 @@ public class TestHBaseFsck { public void testLingeringSplitParent() throws Exception { TableName table = TableName.valueOf("testLingeringSplitParent"); - HTable meta = null; + Table meta = null; try { setupTable(table); assertEquals(ROWKEYS.length, countRows()); @@ -1375,7 +1376,7 @@ public class TestHBaseFsck { public void testValidLingeringSplitParent() throws Exception { TableName table = TableName.valueOf("testLingeringSplitParent"); - HTable meta = null; + Table meta = null; try { setupTable(table); assertEquals(ROWKEYS.length, countRows()); @@ -1425,7 +1426,7 @@ public class TestHBaseFsck { public void testSplitDaughtersNotInMeta() throws Exception { TableName table = TableName.valueOf("testSplitdaughtersNotInMeta"); - HTable meta = null; + Table meta = null; try { setupTable(table); assertEquals(ROWKEYS.length, countRows()); @@ -1977,7 +1978,7 @@ public class TestHBaseFsck { // Mess it up by removing the RegionInfo for one region. final List deletes = new LinkedList(); - HTable meta = new HTable(conf, HTableDescriptor.META_TABLEDESC.getTableName()); + Table meta = new HTable(conf, HTableDescriptor.META_TABLEDESC.getTableName()); MetaScanner.metaScan(conf, new MetaScanner.MetaScannerVisitor() { @Override @@ -2281,7 +2282,7 @@ public class TestHBaseFsck { @Test public void testHbckAfterRegionMerge() throws Exception { TableName table = TableName.valueOf("testMergeRegionFilesInHdfs"); - HTable meta = null; + Table meta = null; try { // disable CatalogJanitor TEST_UTIL.getHBaseCluster().getMaster().setCatalogJanitorEnabled(false); diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/util/TestHBaseFsckEncryption.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/util/TestHBaseFsckEncryption.java index d4c03bbcd88..6dfe90cda50 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/util/TestHBaseFsckEncryption.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/util/TestHBaseFsckEncryption.java @@ -38,6 +38,7 @@ import org.apache.hadoop.hbase.LargeTests; import org.apache.hadoop.hbase.TableName; import org.apache.hadoop.hbase.client.HTable; import org.apache.hadoop.hbase.client.Put; +import org.apache.hadoop.hbase.client.Table; import org.apache.hadoop.hbase.io.crypto.Encryption; import org.apache.hadoop.hbase.io.crypto.KeyProviderForTesting; import org.apache.hadoop.hbase.io.crypto.aes.AES; @@ -101,7 +102,7 @@ public class TestHBaseFsckEncryption { @Test public void testFsckWithEncryption() throws Exception { // Populate the table with some data - HTable table = new HTable(conf, htd.getName()); + Table table = new HTable(conf, htd.getName()); try { byte[] values = { 'A', 'B', 'C', 'D' }; for (int i = 0; i < values.length; i++) { diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/util/TestMergeTable.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/util/TestMergeTable.java index 96a9c3355c7..e27c7c269b2 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/util/TestMergeTable.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/util/TestMergeTable.java @@ -30,6 +30,7 @@ import org.apache.hadoop.fs.FileSystem; import org.apache.hadoop.fs.Path; import org.apache.hadoop.hbase.*; import org.apache.hadoop.hbase.MetaTableAccessor; +import org.apache.hadoop.hbase.client.Admin; import org.apache.hadoop.hbase.client.HBaseAdmin; import org.apache.hadoop.hbase.client.HConnection; import org.apache.hadoop.hbase.client.HConnectionManager; @@ -119,7 +120,7 @@ public class TestMergeTable { desc.getTableName()); LOG.info("originalTableRegions size=" + originalTableRegions.size() + "; " + originalTableRegions); - HBaseAdmin admin = new HBaseAdmin(c); + Admin admin = new HBaseAdmin(c); admin.disableTable(desc.getTableName()); HMerge.merge(c, FileSystem.get(c), desc.getTableName()); List postMergeTableRegions = diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/util/TestMiniClusterLoadSequential.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/util/TestMiniClusterLoadSequential.java index 90cdca2c4fe..d6d0ab7624d 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/util/TestMiniClusterLoadSequential.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/util/TestMiniClusterLoadSequential.java @@ -33,6 +33,7 @@ import org.apache.hadoop.hbase.HConstants; import org.apache.hadoop.hbase.HTableDescriptor; import org.apache.hadoop.hbase.LargeTests; 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.io.compress.Compression; import org.apache.hadoop.hbase.io.encoding.DataBlockEncoding; @@ -150,7 +151,7 @@ public class TestMiniClusterLoadSequential { LOG.info("Starting load test: dataBlockEncoding=" + dataBlockEncoding + ", isMultiPut=" + isMultiPut); numKeys = numKeys(); - HBaseAdmin admin = new HBaseAdmin(conf); + Admin admin = new HBaseAdmin(conf); while (admin.getClusterStatus().getServers().size() < NUM_RS) { LOG.info("Sleeping until " + NUM_RS + " RSs are online"); Threads.sleepWithoutInterrupt(1000); @@ -172,7 +173,7 @@ public class TestMiniClusterLoadSequential { return 1000; } - protected HColumnDescriptor getColumnDesc(HBaseAdmin admin) + protected HColumnDescriptor getColumnDesc(Admin admin) throws TableNotFoundException, IOException { return admin.getTableDescriptor(TABLE).getFamily(CF); } diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/util/TestProcessBasedCluster.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/util/TestProcessBasedCluster.java index 88144a75e43..38f93ae85ba 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/util/TestProcessBasedCluster.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/util/TestProcessBasedCluster.java @@ -31,6 +31,7 @@ import org.apache.hadoop.hbase.client.HTable; import org.apache.hadoop.hbase.client.Result; import org.apache.hadoop.hbase.client.ResultScanner; import org.apache.hadoop.hbase.MediumTests; +import org.apache.hadoop.hbase.client.Table; import org.junit.Test; import org.junit.experimental.categories.Category; @@ -60,7 +61,7 @@ public class TestProcessBasedCluster { HTestConst.DEFAULT_CF_STR_SET, HColumnDescriptor.DEFAULT_VERSIONS, COLS_PER_ROW, FLUSHES, NUM_REGIONS, ROWS_PER_FLUSH); - HTable table = new HTable(TEST_UTIL.getConfiguration(), HTestConst.DEFAULT_TABLE_BYTES); + Table table = new HTable(TEST_UTIL.getConfiguration(), HTestConst.DEFAULT_TABLE_BYTES); ResultScanner scanner = table.getScanner(HTestConst.DEFAULT_CF_BYTES); Result result; int rows = 0; diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/util/TestRegionSizeCalculator.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/util/TestRegionSizeCalculator.java index 6f8a9b334ca..6a3041e4cab 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/util/TestRegionSizeCalculator.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/util/TestRegionSizeCalculator.java @@ -24,6 +24,7 @@ import org.apache.hadoop.hbase.RegionLoad; import org.apache.hadoop.hbase.ServerLoad; import org.apache.hadoop.hbase.ServerName; import org.apache.hadoop.hbase.SmallTests; +import org.apache.hadoop.hbase.client.Admin; import org.apache.hadoop.hbase.client.HBaseAdmin; import org.apache.hadoop.hbase.client.HTable; import org.junit.Test; @@ -52,7 +53,7 @@ public class TestRegionSizeCalculator { HTable table = mockTable("region1", "region2", "region3"); - HBaseAdmin admin = mockAdmin( + Admin admin = mockAdmin( mockServer( mockRegion("region1", 123), mockRegion("region3", 1232) @@ -84,7 +85,7 @@ public class TestRegionSizeCalculator { HTable table = mockTable("largeRegion"); - HBaseAdmin admin = mockAdmin( + Admin admin = mockAdmin( mockServer( mockRegion("largeRegion", Integer.MAX_VALUE) ) @@ -101,7 +102,7 @@ public class TestRegionSizeCalculator { String regionName = "cz.goout:/index.html"; HTable table = mockTable(regionName); - HBaseAdmin admin = mockAdmin( + Admin admin = mockAdmin( mockServer( mockRegion(regionName, 999) ) @@ -141,9 +142,9 @@ public class TestRegionSizeCalculator { /** * Creates mock returing ClusterStatus info about given servers. */ - private HBaseAdmin mockAdmin(ServerLoad... servers) throws Exception { + private Admin mockAdmin(ServerLoad... servers) throws Exception { //get clusterstatus - HBaseAdmin mockAdmin = Mockito.mock(HBaseAdmin.class); + Admin mockAdmin = Mockito.mock(HBaseAdmin.class); ClusterStatus clusterStatus = mockCluster(servers); when(mockAdmin.getClusterStatus()).thenReturn(clusterStatus); return mockAdmin; diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/util/hbck/OfflineMetaRebuildTestCore.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/util/hbck/OfflineMetaRebuildTestCore.java index b6785b23be1..fb52bb2ea97 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/util/hbck/OfflineMetaRebuildTestCore.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/util/hbck/OfflineMetaRebuildTestCore.java @@ -49,6 +49,7 @@ import org.apache.hadoop.hbase.client.Put; import org.apache.hadoop.hbase.client.Result; import org.apache.hadoop.hbase.client.ResultScanner; import org.apache.hadoop.hbase.client.Scan; +import org.apache.hadoop.hbase.client.Table; import org.apache.hadoop.hbase.regionserver.HRegionFileSystem; import org.apache.hadoop.hbase.util.Bytes; import org.apache.hadoop.hbase.util.FSUtils; @@ -136,7 +137,7 @@ public class OfflineMetaRebuildTestCore { } } - private void populateTable(HTable tbl) throws IOException { + private void populateTable(Table tbl) throws IOException { byte[] values = { 'A', 'B', 'C', 'D' }; for (int i = 0; i < values.length; i++) { for (int j = 0; j < values.length; j++) { @@ -190,7 +191,7 @@ public class OfflineMetaRebuildTestCore { hri.getEncodedName()); fs.delete(p, true); - HTable meta = new HTable(conf, TableName.META_TABLE_NAME); + Table meta = new HTable(conf, TableName.META_TABLE_NAME); Delete delete = new Delete(deleteRow); meta.delete(delete); meta.close(); @@ -203,9 +204,9 @@ public class OfflineMetaRebuildTestCore { dumpMeta(htd); } - protected HRegionInfo createRegion(Configuration conf, final HTable htbl, + protected HRegionInfo createRegion(Configuration conf, final Table htbl, byte[] startKey, byte[] endKey) throws IOException { - HTable meta = new HTable(conf, TableName.META_TABLE_NAME); + Table meta = new HTable(conf, TableName.META_TABLE_NAME); HTableDescriptor htd = htbl.getTableDescriptor(); HRegionInfo hri = new HRegionInfo(htbl.getName(), startKey, endKey); @@ -230,7 +231,7 @@ public class OfflineMetaRebuildTestCore { // Mess it up by blowing up meta. Admin admin = TEST_UTIL.getHBaseAdmin(); Scan s = new Scan(); - HTable meta = new HTable(conf, TableName.META_TABLE_NAME); + Table meta = new HTable(conf, TableName.META_TABLE_NAME); ResultScanner scanner = meta.getScanner(s); List dels = new ArrayList(); for (Result r : scanner) { @@ -257,7 +258,7 @@ public class OfflineMetaRebuildTestCore { */ protected int tableRowCount(Configuration conf, TableName table) throws IOException { - HTable t = new HTable(conf, table); + Table t = new HTable(conf, table); Scan st = new Scan(); ResultScanner rst = t.getScanner(st); diff --git a/hbase-thrift/src/main/java/org/apache/hadoop/hbase/thrift/IncrementCoalescer.java b/hbase-thrift/src/main/java/org/apache/hadoop/hbase/thrift/IncrementCoalescer.java index bb45c880a5e..bdbe4456b9c 100644 --- a/hbase-thrift/src/main/java/org/apache/hadoop/hbase/thrift/IncrementCoalescer.java +++ b/hbase-thrift/src/main/java/org/apache/hadoop/hbase/thrift/IncrementCoalescer.java @@ -35,7 +35,7 @@ import java.util.concurrent.atomic.AtomicLong; import org.apache.commons.logging.Log; import org.apache.commons.logging.LogFactory; import org.apache.hadoop.hbase.KeyValue; -import org.apache.hadoop.hbase.client.HTable; +import org.apache.hadoop.hbase.client.Table; import org.apache.hadoop.hbase.thrift.ThriftServerRunner.HBaseHandler; import org.apache.hadoop.hbase.thrift.generated.TIncrement; import org.apache.hadoop.hbase.util.Bytes; @@ -265,7 +265,7 @@ public class IncrementCoalescer implements IncrementCoalescerMBean { continue; } try { - HTable table = handler.getTable(row.getTable()); + Table table = handler.getTable(row.getTable()); if (failures > 2) { throw new IOException("Auto-Fail rest of ICVs"); } diff --git a/hbase-thrift/src/main/java/org/apache/hadoop/hbase/thrift2/ThriftHBaseServiceHandler.java b/hbase-thrift/src/main/java/org/apache/hadoop/hbase/thrift2/ThriftHBaseServiceHandler.java index b13aff99518..e98163f7cba 100644 --- a/hbase-thrift/src/main/java/org/apache/hadoop/hbase/thrift2/ThriftHBaseServiceHandler.java +++ b/hbase-thrift/src/main/java/org/apache/hadoop/hbase/thrift2/ThriftHBaseServiceHandler.java @@ -44,6 +44,7 @@ import org.apache.hadoop.hbase.client.HTableFactory; import org.apache.hadoop.hbase.client.HTableInterface; import org.apache.hadoop.hbase.client.HTablePool; import org.apache.hadoop.hbase.client.ResultScanner; +import org.apache.hadoop.hbase.client.Table; import org.apache.hadoop.hbase.security.UserProvider; import org.apache.hadoop.hbase.thrift.ThriftMetrics; import org.apache.hadoop.hbase.thrift2.generated.*; @@ -143,7 +144,7 @@ public class ThriftHBaseServiceHandler implements THBaseService.Iface { }; } - private HTableInterface getTable(ByteBuffer tableName) { + private Table getTable(ByteBuffer tableName) { String currentUser = connectionCache.getEffectiveUser(); try { HTablePool htablePool = htablePools.get(currentUser, htablePoolCreater); @@ -153,7 +154,7 @@ public class ThriftHBaseServiceHandler implements THBaseService.Iface { } } - private void closeTable(HTableInterface table) throws TIOError { + private void closeTable(Table table) throws TIOError { try { table.close(); } catch (IOException e) { @@ -202,7 +203,7 @@ public class ThriftHBaseServiceHandler implements THBaseService.Iface { @Override public boolean exists(ByteBuffer table, TGet get) throws TIOError, TException { - HTableInterface htable = getTable(table); + Table htable = getTable(table); try { return htable.exists(getFromThrift(get)); } catch (IOException e) { @@ -214,7 +215,7 @@ public class ThriftHBaseServiceHandler implements THBaseService.Iface { @Override public TResult get(ByteBuffer table, TGet get) throws TIOError, TException { - HTableInterface htable = getTable(table); + Table htable = getTable(table); try { return resultFromHBase(htable.get(getFromThrift(get))); } catch (IOException e) { @@ -226,7 +227,7 @@ public class ThriftHBaseServiceHandler implements THBaseService.Iface { @Override public List getMultiple(ByteBuffer table, List gets) throws TIOError, TException { - HTableInterface htable = getTable(table); + Table htable = getTable(table); try { return resultsFromHBase(htable.get(getsFromThrift(gets))); } catch (IOException e) { @@ -238,7 +239,7 @@ public class ThriftHBaseServiceHandler implements THBaseService.Iface { @Override public void put(ByteBuffer table, TPut put) throws TIOError, TException { - HTableInterface htable = getTable(table); + Table htable = getTable(table); try { htable.put(putFromThrift(put)); } catch (IOException e) { @@ -251,7 +252,7 @@ public class ThriftHBaseServiceHandler implements THBaseService.Iface { @Override public boolean checkAndPut(ByteBuffer table, ByteBuffer row, ByteBuffer family, ByteBuffer qualifier, ByteBuffer value, TPut put) throws TIOError, TException { - HTableInterface htable = getTable(table); + Table htable = getTable(table); try { return htable.checkAndPut(byteBufferToByteArray(row), byteBufferToByteArray(family), byteBufferToByteArray(qualifier), (value == null) ? null : byteBufferToByteArray(value), @@ -265,7 +266,7 @@ public class ThriftHBaseServiceHandler implements THBaseService.Iface { @Override public void putMultiple(ByteBuffer table, List puts) throws TIOError, TException { - HTableInterface htable = getTable(table); + Table htable = getTable(table); try { htable.put(putsFromThrift(puts)); } catch (IOException e) { @@ -277,7 +278,7 @@ public class ThriftHBaseServiceHandler implements THBaseService.Iface { @Override public void deleteSingle(ByteBuffer table, TDelete deleteSingle) throws TIOError, TException { - HTableInterface htable = getTable(table); + Table htable = getTable(table); try { htable.delete(deleteFromThrift(deleteSingle)); } catch (IOException e) { @@ -290,7 +291,7 @@ public class ThriftHBaseServiceHandler implements THBaseService.Iface { @Override public List deleteMultiple(ByteBuffer table, List deletes) throws TIOError, TException { - HTableInterface htable = getTable(table); + Table htable = getTable(table); try { htable.delete(deletesFromThrift(deletes)); } catch (IOException e) { @@ -304,7 +305,7 @@ public class ThriftHBaseServiceHandler implements THBaseService.Iface { @Override public boolean checkAndDelete(ByteBuffer table, ByteBuffer row, ByteBuffer family, ByteBuffer qualifier, ByteBuffer value, TDelete deleteSingle) throws TIOError, TException { - HTableInterface htable = getTable(table); + Table htable = getTable(table); try { if (value == null) { @@ -324,7 +325,7 @@ public class ThriftHBaseServiceHandler implements THBaseService.Iface { @Override public TResult increment(ByteBuffer table, TIncrement increment) throws TIOError, TException { - HTableInterface htable = getTable(table); + Table htable = getTable(table); try { return resultFromHBase(htable.increment(incrementFromThrift(increment))); } catch (IOException e) { @@ -336,7 +337,7 @@ public class ThriftHBaseServiceHandler implements THBaseService.Iface { @Override public TResult append(ByteBuffer table, TAppend append) throws TIOError, TException { - HTableInterface htable = getTable(table); + Table htable = getTable(table); try { return resultFromHBase(htable.append(appendFromThrift(append))); } catch (IOException e) { @@ -348,7 +349,7 @@ public class ThriftHBaseServiceHandler implements THBaseService.Iface { @Override public int openScanner(ByteBuffer table, TScan scan) throws TIOError, TException { - HTableInterface htable = getTable(table); + Table htable = getTable(table); ResultScanner resultScanner = null; try { resultScanner = htable.getScanner(scanFromThrift(scan)); @@ -380,7 +381,7 @@ public class ThriftHBaseServiceHandler implements THBaseService.Iface { @Override public List getScannerResults(ByteBuffer table, TScan scan, int numRows) throws TIOError, TException { - HTableInterface htable = getTable(table); + Table htable = getTable(table); List results = null; ResultScanner scanner = null; try { @@ -414,7 +415,7 @@ public class ThriftHBaseServiceHandler implements THBaseService.Iface { @Override public void mutateRow(ByteBuffer table, TRowMutations rowMutations) throws TIOError, TException { - HTableInterface htable = getTable(table); + Table htable = getTable(table); try { htable.mutateRow(rowMutationsFromThrift(rowMutations)); } catch (IOException e) { diff --git a/hbase-thrift/src/test/java/org/apache/hadoop/hbase/thrift2/TestThriftHBaseServiceHandler.java b/hbase-thrift/src/test/java/org/apache/hadoop/hbase/thrift2/TestThriftHBaseServiceHandler.java index 5227df4c12a..22236efae2f 100644 --- a/hbase-thrift/src/test/java/org/apache/hadoop/hbase/thrift2/TestThriftHBaseServiceHandler.java +++ b/hbase-thrift/src/test/java/org/apache/hadoop/hbase/thrift2/TestThriftHBaseServiceHandler.java @@ -27,6 +27,7 @@ import org.apache.hadoop.hbase.HColumnDescriptor; import org.apache.hadoop.hbase.HTableDescriptor; import org.apache.hadoop.hbase.MediumTests; import org.apache.hadoop.hbase.TableName; +import org.apache.hadoop.hbase.client.Admin; import org.apache.hadoop.hbase.client.Get; import org.apache.hadoop.hbase.client.HBaseAdmin; import org.apache.hadoop.hbase.client.Put; @@ -133,7 +134,7 @@ public class TestThriftHBaseServiceHandler { @BeforeClass public static void beforeClass() throws Exception { UTIL.startMiniCluster(); - HBaseAdmin admin = new HBaseAdmin(UTIL.getConfiguration()); + Admin admin = new HBaseAdmin(UTIL.getConfiguration()); HTableDescriptor tableDescriptor = new HTableDescriptor(TableName.valueOf(tableAname)); for (HColumnDescriptor family : families) { tableDescriptor.addFamily(family); diff --git a/hbase-thrift/src/test/java/org/apache/hadoop/hbase/thrift2/TestThriftHBaseServiceHandlerWithLabels.java b/hbase-thrift/src/test/java/org/apache/hadoop/hbase/thrift2/TestThriftHBaseServiceHandlerWithLabels.java index 29cb4ce551f..765b3b44b3f 100644 --- a/hbase-thrift/src/test/java/org/apache/hadoop/hbase/thrift2/TestThriftHBaseServiceHandlerWithLabels.java +++ b/hbase-thrift/src/test/java/org/apache/hadoop/hbase/thrift2/TestThriftHBaseServiceHandlerWithLabels.java @@ -39,6 +39,7 @@ import org.apache.hadoop.hbase.HColumnDescriptor; import org.apache.hadoop.hbase.HTableDescriptor; import org.apache.hadoop.hbase.MediumTests; import org.apache.hadoop.hbase.TableName; +import org.apache.hadoop.hbase.client.Admin; import org.apache.hadoop.hbase.client.HBaseAdmin; import org.apache.hadoop.hbase.protobuf.generated.VisibilityLabelsProtos.VisibilityLabelsResponse; import org.apache.hadoop.hbase.security.User; @@ -136,7 +137,7 @@ public static void beforeClass() throws Exception { // Wait for the labels table to become available UTIL.waitTableEnabled(VisibilityConstants.LABELS_TABLE_NAME.getName(), 50000); createLabels(); - HBaseAdmin admin = new HBaseAdmin(UTIL.getConfiguration()); + Admin admin = new HBaseAdmin(UTIL.getConfiguration()); HTableDescriptor tableDescriptor = new HTableDescriptor( TableName.valueOf(tableAname)); for (HColumnDescriptor family : families) {