From 9dbe525c1156804c06fefadc461be01a5b70f9f1 Mon Sep 17 00:00:00 2001
From: Jurriaan Mous
Date: Sun, 29 May 2016 17:38:57 +0200
Subject: [PATCH] HBASE-15875 Remove HTable references and HTableInterface
---
bin/region_status.rb | 2 +-
.../hadoop/hbase/client/BufferedMutator.java | 2 +-
.../hadoop/hbase/client/BufferedMutatorImpl.java | 2 +-
.../hbase/client/ConnectionImplementation.java | 4 +-
.../org/apache/hadoop/hbase/client/HTable.java | 52 +---
.../java/org/apache/hadoop/hbase/client/Table.java | 8 +-
.../hbase/ipc/SyncCoprocessorRpcChannel.java | 2 +-
.../hadoop/hbase/client/TestAsyncProcess.java | 12 +-
.../apache/hadoop/hbase/client/HTableWrapper.java | 16 +-
.../hbase/mapreduce/LoadIncrementalHFiles.java | 4 +-
.../mob/compactions/PartitionedMobCompactor.java | 12 +-
.../main/resources/hbase-webapps/master/table.jsp | 10 +-
.../apache/hadoop/hbase/HBaseTestingUtility.java | 126 ++++-----
.../hbase/client/HConnectionTestingUtility.java | 2 +-
.../hbase/client/TestBlockEvictionFromClient.java | 72 ++---
.../hadoop/hbase/client/TestClientPushback.java | 10 +-
.../hadoop/hbase/client/TestFromClientSide.java | 306 +++++++++------------
.../hbase/client/TestFromClientSideNoCodec.java | 15 +-
.../org/apache/hadoop/hbase/client/TestHCM.java | 189 +++++++------
.../client/TestHTableMultiplexerFlushCache.java | 2 +-
.../hadoop/hbase/client/TestLeaseRenewal.java | 2 +-
.../coprocessor/TestRegionObserverInterface.java | 3 +-
.../hbase/mapreduce/TestHFileOutputFormat2.java | 9 +-
.../TestSimpleRegionNormalizerOnCluster.java | 6 +-
.../regionserver/TestCorruptedRegionStoreFile.java | 7 +-
.../regionserver/TestRegionServerMetrics.java | 3 +-
.../TestFlushWithThroughputController.java | 3 +-
.../hbase/regionserver/wal/TestLogRolling.java | 3 -
.../security/token/TestTokenAuthentication.java | 6 +-
.../apache/hadoop/hbase/tool/TestCanaryTool.java | 10 +-
.../hadoop/hbase/util/MultiThreadedReader.java | 1 -
.../hbase/util/MultiThreadedReaderWithACL.java | 3 +-
.../hadoop/hbase/util/MultiThreadedUpdater.java | 1 -
.../hbase/util/MultiThreadedUpdaterWithACL.java | 3 +-
.../hadoop/hbase/util/MultiThreadedWriter.java | 1 -
.../hbase/util/MultiThreadedWriterWithACL.java | 3 +-
.../hbase/thrift2/ThriftHBaseServiceHandler.java | 2 +-
37 files changed, 403 insertions(+), 511 deletions(-)
diff --git a/bin/region_status.rb b/bin/region_status.rb
index 55bc672..91873cb 100644
--- a/bin/region_status.rb
+++ b/bin/region_status.rb
@@ -50,7 +50,7 @@ import org.apache.hadoop.hbase.TableName
import org.apache.hadoop.hbase.HConstants
import org.apache.hadoop.hbase.MasterNotRunningException
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.Scan
import org.apache.hadoop.hbase.filter.FirstKeyOnlyFilter
import org.apache.hadoop.hbase.util.Bytes
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/BufferedMutator.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/BufferedMutator.java
index 3287335..5dc7fc3 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/BufferedMutator.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/BufferedMutator.java
@@ -52,7 +52,7 @@ import java.util.List;
* extreme circumstances, such as JVM or machine failure, may cause some data loss.
*
* NOTE: This class replaces the functionality that used to be available via
- *HTableInterface#setAutoFlush(boolean) set to {@code false}.
+ * HTable#setAutoFlush(boolean) set to {@code false}.
*
*
* See also the {@code BufferedMutatorExample} in the hbase-examples module.
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/BufferedMutatorImpl.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/BufferedMutatorImpl.java
index 01aaec5..2a7effe 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/BufferedMutatorImpl.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/BufferedMutatorImpl.java
@@ -260,7 +260,7 @@ public class BufferedMutatorImpl implements BufferedMutator {
/**
* This is used for legacy purposes in {@link HTable#setWriteBufferSize(long)} only. This ought
* not be called for production uses.
- * @deprecated Going away when we drop public support for {@link HTableInterface}.
+ * @deprecated Going away when we drop public support for {@link HTable}.
*/
@Deprecated
public void setWriteBufferSize(long writeBufferSize) throws RetriesExhaustedWithDetailsException,
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ConnectionImplementation.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ConnectionImplementation.java
index 0382a71..d93a8b4 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ConnectionImplementation.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ConnectionImplementation.java
@@ -275,12 +275,12 @@ class ConnectionImplementation implements ClusterConnection, Closeable {
}
@Override
- public HTableInterface getTable(TableName tableName) throws IOException {
+ public Table getTable(TableName tableName) throws IOException {
return getTable(tableName, getBatchPool());
}
@Override
- public HTableInterface getTable(TableName tableName, ExecutorService pool) throws IOException {
+ public Table getTable(TableName tableName, ExecutorService pool) throws IOException {
return new HTable(tableName, this, connectionConfig,
rpcCallerFactory, rpcControllerFactory, pool);
}
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 60a12a4..54fbfe9 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
@@ -100,14 +100,13 @@ import org.apache.hadoop.hbase.util.Threads;
*/
@InterfaceAudience.Private
@InterfaceStability.Stable
-public class HTable implements HTableInterface {
+public class HTable implements Table {
private static final Log LOG = LogFactory.getLog(HTable.class);
protected ClusterConnection connection;
private final TableName tableName;
private volatile Configuration configuration;
private ConnectionConfiguration connConfiguration;
protected BufferedMutatorImpl mutator;
- private boolean autoFlush = true;
private boolean closed = false;
protected int scannerCaching;
protected long scannerMaxResultSize;
@@ -237,14 +236,6 @@ public class HTable implements HTableInterface {
return configuration;
}
- /**
- * {@inheritDoc}
- */
- @Override
- public byte [] getTableName() {
- return this.tableName.getName();
- }
-
@Override
public TableName getName() {
return tableName;
@@ -328,7 +319,7 @@ public class HTable implements HTableInterface {
/**
* The underlying {@link HTable} must not be closed.
- * {@link HTableInterface#getScanner(Scan)} has other usage details.
+ * {@link Table#getScanner(Scan)} has other usage details.
*/
@Override
public ResultScanner getScanner(final Scan scan) throws IOException {
@@ -379,7 +370,7 @@ public class HTable implements HTableInterface {
/**
* The underlying {@link HTable} must not be closed.
- * {@link HTableInterface#getScanner(byte[])} has other usage details.
+ * {@link Table#getScanner(byte[])} has other usage details.
*/
@Override
public ResultScanner getScanner(byte [] family) throws IOException {
@@ -390,7 +381,7 @@ public class HTable implements HTableInterface {
/**
* The underlying {@link HTable} must not be closed.
- * {@link HTableInterface#getScanner(byte[], byte[])} has other usage details.
+ * {@link Table#getScanner(byte[], byte[])} has other usage details.
*/
@Override
public ResultScanner getScanner(byte [] family, byte [] qualifier)
@@ -572,9 +563,7 @@ public class HTable implements HTableInterface {
@Override
public void put(final Put put) throws IOException {
getBufferedMutator().mutate(put);
- if (autoFlush) {
- flushCommits();
- }
+ flushCommits();
}
/**
@@ -584,9 +573,7 @@ public class HTable implements HTableInterface {
@Override
public void put(final List puts) throws IOException {
getBufferedMutator().mutate(puts);
- if (autoFlush) {
- flushCommits();
- }
+ flushCommits();
}
/**
@@ -984,8 +971,7 @@ public class HTable implements HTableInterface {
* {@inheritDoc}
* @throws IOException
*/
- @Override
- public void flushCommits() throws IOException {
+ void flushCommits() throws IOException {
if (mutator == null) {
// nothing to flush if there's no mutator; don't bother creating one.
return;
@@ -1070,30 +1056,6 @@ public class HTable implements HTableInterface {
}
/**
- * {@inheritDoc}
- */
- @Override
- public boolean isAutoFlush() {
- return autoFlush;
- }
-
- /**
- * {@inheritDoc}
- */
- @Override
- public void setAutoFlushTo(boolean autoFlush) {
- this.autoFlush = autoFlush;
- }
-
- /**
- * {@inheritDoc}
- */
- @Override
- public void setAutoFlush(boolean autoFlush, boolean clearBufferOnFail) {
- this.autoFlush = autoFlush;
- }
-
- /**
* Returns the maximum size in bytes of the write buffer for this HTable.
*
* The default value comes from the configuration parameter
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/Table.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/Table.java
index 1b0f387..f2cec97 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/Table.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/Table.java
@@ -577,12 +577,12 @@ public interface Table extends Closeable {
* early and throw SocketTimeoutException.
* @param operationTimeout the total timeout of each operation in millisecond.
*/
- public void setOperationTimeout(int operationTimeout);
+ void setOperationTimeout(int operationTimeout);
/**
* Get timeout (millisecond) of each operation for in Table instance.
*/
- public int getOperationTimeout();
+ int getOperationTimeout();
/**
* Set timeout (millisecond) of each rpc request in operations of this Table instance, will
@@ -591,11 +591,11 @@ public interface Table extends Closeable {
* retries exhausted or operation timeout reached.
* @param rpcTimeout the timeout of each rpc request in millisecond.
*/
- public void setRpcTimeout(int rpcTimeout);
+ void setRpcTimeout(int rpcTimeout);
/**
* Get timeout (millisecond) of each rpc request in this Table instance.
*/
- public int getRpcTimeout();
+ int getRpcTimeout();
}
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/ipc/SyncCoprocessorRpcChannel.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/ipc/SyncCoprocessorRpcChannel.java
index af8ddd4..347d8a1 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/ipc/SyncCoprocessorRpcChannel.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/ipc/SyncCoprocessorRpcChannel.java
@@ -35,7 +35,7 @@ import org.apache.hadoop.hbase.protobuf.ResponseConverter;
* Base class which provides clients with an RPC connection to
* call coprocessor endpoint {@link com.google.protobuf.Service}s.
* Note that clients should not use this class directly, except through
- * {@link org.apache.hadoop.hbase.client.HTableInterface#coprocessorService(byte[])}.
+ * {@link org.apache.hadoop.hbase.client.Table#coprocessorService(byte[])}.
*/
@InterfaceAudience.Public
@InterfaceStability.Evolving
diff --git a/hbase-client/src/test/java/org/apache/hadoop/hbase/client/TestAsyncProcess.java b/hbase-client/src/test/java/org/apache/hadoop/hbase/client/TestAsyncProcess.java
index 376c02a..839a33a 100644
--- a/hbase-client/src/test/java/org/apache/hadoop/hbase/client/TestAsyncProcess.java
+++ b/hbase-client/src/test/java/org/apache/hadoop/hbase/client/TestAsyncProcess.java
@@ -712,15 +712,17 @@ public class TestAsyncProcess {
private void doHTableFailedPut(boolean bufferOn) throws Exception {
ClusterConnection conn = createHConnection();
- HTable ht = new HTable(conn, new BufferedMutatorParams(DUMMY_TABLE));
- MyAsyncProcess ap = new MyAsyncProcess(conn, conf, true);
- ht.mutator.ap = ap;
+ BufferedMutatorParams bufferParam = new BufferedMutatorParams(DUMMY_TABLE);
if (bufferOn) {
- ht.setWriteBufferSize(1024L * 1024L);
+ bufferParam.writeBufferSize(1024L * 1024L);
} else {
- ht.setWriteBufferSize(0L);
+ bufferParam.writeBufferSize(0L);
}
+ HTable ht = new HTable(conn, bufferParam);
+ MyAsyncProcess ap = new MyAsyncProcess(conn, conf, true);
+ ht.mutator.ap = ap;
+
Put put = createPut(1, false);
Assert.assertEquals(0L, ht.mutator.currentWriteBufferSize.get());
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/client/HTableWrapper.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/client/HTableWrapper.java
index 292a935..5da0df7 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/client/HTableWrapper.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/client/HTableWrapper.java
@@ -18,6 +18,11 @@
*/
package org.apache.hadoop.hbase.client;
+import com.google.protobuf.Descriptors.MethodDescriptor;
+import com.google.protobuf.Message;
+import com.google.protobuf.Service;
+import com.google.protobuf.ServiceException;
+
import java.io.IOException;
import java.util.ArrayList;
import java.util.List;
@@ -37,11 +42,6 @@ import org.apache.hadoop.hbase.filter.CompareFilter.CompareOp;
import org.apache.hadoop.hbase.ipc.CoprocessorRpcChannel;
import org.apache.hadoop.io.MultipleIOException;
-import com.google.protobuf.Descriptors.MethodDescriptor;
-import com.google.protobuf.Message;
-import com.google.protobuf.Service;
-import com.google.protobuf.ServiceException;
-
/**
* A wrapper for HTable. Can be used to restrict privilege.
*
@@ -61,7 +61,7 @@ import com.google.protobuf.ServiceException;
public final class HTableWrapper implements Table {
private final Table table;
- private ClusterConnection connection;
+ private final ClusterConnection connection;
private final List
openTables;
/**
@@ -134,7 +134,9 @@ public final class HTableWrapper implements Table {
public Boolean[] exists(List gets) throws IOException {
// Do convertion.
boolean [] exists = table.existsAll(gets);
- if (exists == null) return null;
+ if (exists == null) {
+ return null;
+ }
Boolean [] results = new Boolean [exists.length];
for (int i = 0; i < exists.length; i++) {
results[i] = exists[i]? Boolean.TRUE: Boolean.FALSE;
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 5781a42..a23d739 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
@@ -444,7 +444,7 @@ public class LoadIncrementalHFiles extends Configured implements Tool {
private void validateFamiliesInHFiles(Table table, Deque queue)
throws IOException {
Collection families = table.getTableDescriptor().getFamilies();
- List familyNames = new ArrayList(families.size());
+ List familyNames = new ArrayList<>(families.size());
for (HColumnDescriptor family : families) {
familyNames.add(family.getNameAsString());
}
@@ -501,7 +501,7 @@ public class LoadIncrementalHFiles extends Configured implements Tool {
ExecutorService pool, Deque queue,
final Multimap regionGroups) throws IOException {
// atomically bulk load the groups.
- Set>> loadingFutures = new HashSet>>();
+ Set>> loadingFutures = new HashSet<>();
for (Entry> e: regionGroups.asMap().entrySet()){
final byte[] first = e.getKey().array();
final Collection lqis = e.getValue();
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/mob/compactions/PartitionedMobCompactor.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/mob/compactions/PartitionedMobCompactor.java
index b4d4bab..29b7e8a 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/mob/compactions/PartitionedMobCompactor.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/mob/compactions/PartitionedMobCompactor.java
@@ -396,7 +396,7 @@ public class PartitionedMobCompactor extends MobCompactor {
long mobCells = 0;
try {
writer = MobUtils.createWriter(conf, fs, column, partition.getPartitionId().getDate(),
- tempPath, Long.MAX_VALUE, column.getCompactionCompression(), partition.getPartitionId()
+ tempPath, Long.MAX_VALUE, column.getCompactionCompressionType(), partition.getPartitionId()
.getStartKey(), compactionCacheConfig, cryptoContext);
filePath = writer.getPath();
byte[] fileName = Bytes.toBytes(filePath.getName());
@@ -508,7 +508,7 @@ public class PartitionedMobCompactor extends MobCompactor {
try {
writer = MobUtils.createDelFileWriter(conf, fs, column,
MobUtils.formatDate(new Date(request.selectionTime)), tempPath, Long.MAX_VALUE,
- column.getCompactionCompression(), HConstants.EMPTY_START_ROW, compactionCacheConfig,
+ column.getCompactionCompressionType(), HConstants.EMPTY_START_ROW, compactionCacheConfig,
cryptoContext);
filePath = writer.getPath();
List cells = new ArrayList<>();
@@ -558,9 +558,8 @@ public class PartitionedMobCompactor extends MobCompactor {
scan.setMaxVersions(column.getMaxVersions());
long ttl = HStore.determineTTLFromFamily(column);
ScanInfo scanInfo = new ScanInfo(conf, column, ttl, 0, CellComparator.COMPARATOR);
- StoreScanner scanner = new StoreScanner(scan, scanInfo, scanType, null, scanners, 0L,
+ return new StoreScanner(scan, scanInfo, scanType, null, scanners, 0L,
HConstants.LATEST_TIMESTAMP);
- return scanner;
}
/**
@@ -649,7 +648,7 @@ public class PartitionedMobCompactor extends MobCompactor {
maxKeyCount += Bytes.toLong(count);
}
}
- return new Pair<>(Long.valueOf(maxSeqId), Long.valueOf(maxKeyCount));
+ return new Pair<>(maxSeqId, maxKeyCount);
}
/**
@@ -680,8 +679,7 @@ public class PartitionedMobCompactor extends MobCompactor {
private FileStatus getFileStatus(Path path) throws IOException {
try {
if (path != null) {
- FileStatus file = fs.getFileStatus(path);
- return file;
+ return fs.getFileStatus(path);
}
} catch (FileNotFoundException e) {
LOG.warn("The file " + path + " can not be found", e);
diff --git a/hbase-server/src/main/resources/hbase-webapps/master/table.jsp b/hbase-server/src/main/resources/hbase-webapps/master/table.jsp
index ee2a7ba..cc13972 100644
--- a/hbase-server/src/main/resources/hbase-webapps/master/table.jsp
+++ b/hbase-server/src/main/resources/hbase-webapps/master/table.jsp
@@ -32,10 +32,6 @@
import="org.owasp.esapi.ESAPI"
import="org.apache.hadoop.conf.Configuration"
import="org.apache.hadoop.util.StringUtils"
- import="org.apache.hadoop.hbase.client.HTable"
- import="org.apache.hadoop.hbase.client.Admin"
- import="org.apache.hadoop.hbase.client.CompactionState"
- import="org.apache.hadoop.hbase.client.RegionLocator"
import="org.apache.hadoop.hbase.HRegionInfo"
import="org.apache.hadoop.hbase.HRegionLocation"
import="org.apache.hadoop.hbase.ServerName"
@@ -50,9 +46,9 @@
import="org.apache.hadoop.hbase.protobuf.generated.HBaseProtos"
import="org.apache.hadoop.hbase.TableName"
import="org.apache.hadoop.hbase.HColumnDescriptor"
- import="org.apache.hadoop.hbase.client.RegionReplicaUtil"
import="org.apache.hadoop.hbase.HBaseConfiguration"
import="org.apache.hadoop.hbase.TableNotFoundException"%>
+<%@ page import="org.apache.hadoop.hbase.client.*" %>
<%
HMaster master = (HMaster)getServletContext().getAttribute(HMaster.MASTER);
Configuration conf = master.getConfiguration();
@@ -64,7 +60,7 @@
final boolean reverseOrder = (reverse==null||!reverse.equals("false"));
String showWholeKey = request.getParameter("showwhole");
final boolean showWhole = (showWholeKey!=null && showWholeKey.equals("true"));
- HTable table = null;
+ Table table;
String tableHeader;
boolean withReplica = false;
ServerName rl = metaTableLocator.getMetaRegionLocation(master.getZooKeeper());
@@ -141,7 +137,7 @@
<%
if ( fqtn != null ) {
try {
- table = (HTable) master.getConnection().getTable(TableName.valueOf(fqtn));
+ table = master.getConnection().getTable(TableName.valueOf(fqtn));
if (table.getTableDescriptor().getRegionReplication() > 1) {
tableHeader = "Table Regions Name Region Server ReadRequests WriteRequests StorefileSize Num.Storefiles MemSize Locality Start Key End Key ReplicaID ";
withReplica = true;
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 b71836b..b618d8c 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
@@ -213,8 +213,8 @@ public class HBaseTestingUtility extends HBaseCommonTestingUtility {
/** This is for unit tests parameterized with a two booleans. */
public static final List BOOLEAN_PARAMETERIZED =
Arrays.asList(new Object[][] {
- { new Boolean(false) },
- { new Boolean(true) }
+ {false},
+ {true}
});
/** This is for unit tests parameterized with a single boolean. */
@@ -263,7 +263,7 @@ public class HBaseTestingUtility extends HBaseCommonTestingUtility {
* testing.
*/
private static List bloomAndCompressionCombinations() {
- List configurations = new ArrayList();
+ List configurations = new ArrayList<>();
for (Compression.Algorithm comprAlgo :
HBaseTestingUtility.COMPRESSION_ALGORITHMS) {
for (BloomType bloomType : BloomType.values()) {
@@ -277,7 +277,7 @@ public class HBaseTestingUtility extends HBaseCommonTestingUtility {
* Create combination of memstoreTS and tags
*/
private static List memStoreTSAndTagsCombination() {
- List configurations = new ArrayList();
+ List configurations = new ArrayList<>();
configurations.add(new Object[] { false, false });
configurations.add(new Object[] { false, true });
configurations.add(new Object[] { true, false });
@@ -286,7 +286,7 @@ public class HBaseTestingUtility extends HBaseCommonTestingUtility {
}
public static List memStoreTSTagsAndOffheapCombination() {
- List configurations = new ArrayList();
+ List configurations = new ArrayList<>();
configurations.add(new Object[] { false, false, true });
configurations.add(new Object[] { false, false, false });
configurations.add(new Object[] { false, true, true });
@@ -542,7 +542,7 @@ public class HBaseTestingUtility extends HBaseCommonTestingUtility {
//file system, the tests should use getBaseTestDir, otherwise, we can use
//the working directory, and create a unique sub dir there
FileSystem fs = getTestFileSystem();
- Path newDataTestDir = null;
+ Path newDataTestDir;
if (fs.getUri().getScheme().equals(FileSystem.getLocal(conf).getUri().getScheme())) {
File dataTestDir = new File(getDataTestDir().toString());
if (deleteOnExit()) dataTestDir.deleteOnExit();
@@ -1084,7 +1084,7 @@ public class HBaseTestingUtility extends HBaseCommonTestingUtility {
s.close();
t.close();
- getHBaseAdmin(); // create immediately the hbaseAdmin
+ getAdmin(); // create immediately the hbaseAdmin
LOG.info("Minicluster is up");
// Set the hbase.fs.tmp.dir config to make sure that we have some default value. This is
@@ -1316,7 +1316,7 @@ public class HBaseTestingUtility extends HBaseCommonTestingUtility {
*/
public Table createTable(TableName tableName, String[] families)
throws IOException {
- List fams = new ArrayList(families.length);
+ List fams = new ArrayList<>(families.length);
for (String family : families) {
fams.add(Bytes.toBytes(family));
}
@@ -1330,7 +1330,7 @@ public class HBaseTestingUtility extends HBaseCommonTestingUtility {
* @return An HTable instance for the created table.
* @throws IOException
*/
- public HTable createTable(TableName tableName, byte[] family)
+ public Table createTable(TableName tableName, byte[] family)
throws IOException{
return createTable(tableName, new byte[][]{family});
}
@@ -1343,7 +1343,7 @@ public class HBaseTestingUtility extends HBaseCommonTestingUtility {
* @return An HTable instance for the created table.
* @throws IOException
*/
- public HTable createMultiRegionTable(TableName tableName, byte[] family, int numRegions)
+ public Table createMultiRegionTable(TableName tableName, byte[] family, int numRegions)
throws IOException {
if (numRegions < 3) throw new IOException("Must create at least 3 regions");
byte[] startKey = Bytes.toBytes("aaaaa");
@@ -1360,7 +1360,7 @@ public class HBaseTestingUtility extends HBaseCommonTestingUtility {
* @return An HTable instance for the created table.
* @throws IOException
*/
- public HTable createTable(TableName tableName, byte[][] families)
+ public Table createTable(TableName tableName, byte[][] families)
throws IOException {
return createTable(tableName, families, (byte[][]) null);
}
@@ -1372,7 +1372,7 @@ public class HBaseTestingUtility extends HBaseCommonTestingUtility {
* @return An HTable instance for the created table.
* @throws IOException
*/
- public HTable createMultiRegionTable(TableName tableName, byte[][] families) throws IOException {
+ public Table createMultiRegionTable(TableName tableName, byte[][] families) throws IOException {
return createTable(tableName, families, KEYS_FOR_HBA_CREATE_TABLE);
}
@@ -1384,12 +1384,12 @@ public class HBaseTestingUtility extends HBaseCommonTestingUtility {
* @return An HTable instance for the created table.
* @throws IOException
*/
- public HTable createTable(TableName tableName, byte[][] families, byte[][] splitKeys)
+ public Table createTable(TableName tableName, byte[][] families, byte[][] splitKeys)
throws IOException {
return createTable(tableName, families, splitKeys, new Configuration(getConfiguration()));
}
- public HTable createTable(TableName tableName, byte[][] families,
+ public Table createTable(TableName tableName, byte[][] families,
int numVersions, byte[] startKey, byte[] endKey, int numRegions)
throws IOException{
HTableDescriptor desc = new HTableDescriptor(tableName);
@@ -1398,11 +1398,11 @@ public class HBaseTestingUtility extends HBaseCommonTestingUtility {
.setMaxVersions(numVersions);
desc.addFamily(hcd);
}
- getHBaseAdmin().createTable(desc, startKey, endKey, numRegions);
+ getAdmin().createTable(desc, startKey, endKey, numRegions);
// HBaseAdmin only waits for regions to appear in hbase:meta we
// should wait until they are assigned
waitUntilAllRegionsAssigned(tableName);
- return (HTable) getConnection().getTable(tableName);
+ return getConnection().getTable(tableName);
}
/**
@@ -1413,9 +1413,9 @@ public class HBaseTestingUtility extends HBaseCommonTestingUtility {
* @return An HTable instance for the created table.
* @throws IOException
*/
- public HTable createTable(HTableDescriptor htd, byte[][] families, Configuration c)
+ public Table createTable(HTableDescriptor htd, byte[][] families, Configuration c)
throws IOException {
- return createTable(htd, families, (byte[][]) null, c);
+ return createTable(htd, families, null, c);
}
/**
@@ -1427,7 +1427,7 @@ public class HBaseTestingUtility extends HBaseCommonTestingUtility {
* @return An HTable instance for the created table.
* @throws IOException
*/
- public HTable createTable(HTableDescriptor htd, byte[][] families, byte[][] splitKeys,
+ public Table createTable(HTableDescriptor htd, byte[][] families, byte[][] splitKeys,
Configuration c) throws IOException {
for (byte[] family : families) {
HColumnDescriptor hcd = new HColumnDescriptor(family);
@@ -1437,11 +1437,11 @@ public class HBaseTestingUtility extends HBaseCommonTestingUtility {
hcd.setBloomFilterType(BloomType.NONE);
htd.addFamily(hcd);
}
- getHBaseAdmin().createTable(htd, splitKeys);
+ getAdmin().createTable(htd, splitKeys);
// HBaseAdmin only waits for regions to appear in hbase:meta
// we should wait until they are assigned
waitUntilAllRegionsAssigned(htd.getTableName());
- return (HTable) getConnection().getTable(htd.getTableName());
+ return getConnection().getTable(htd.getTableName());
}
/**
@@ -1451,13 +1451,13 @@ public class HBaseTestingUtility extends HBaseCommonTestingUtility {
* @return An HTable instance for the created table.
* @throws IOException
*/
- public HTable createTable(HTableDescriptor htd, byte[][] splitRows)
+ public Table createTable(HTableDescriptor htd, byte[][] splitRows)
throws IOException {
- getHBaseAdmin().createTable(htd, splitRows);
+ getAdmin().createTable(htd, splitRows);
// HBaseAdmin only waits for regions to appear in hbase:meta
// we should wait until they are assigned
waitUntilAllRegionsAssigned(htd.getTableName());
- return (HTable) getConnection().getTable(htd.getTableName());
+ return getConnection().getTable(htd.getTableName());
}
/**
@@ -1469,7 +1469,7 @@ public class HBaseTestingUtility extends HBaseCommonTestingUtility {
* @return An HTable instance for the created table.
* @throws IOException
*/
- public HTable createTable(TableName tableName, byte[][] families, byte[][] splitKeys,
+ public Table createTable(TableName tableName, byte[][] families, byte[][] splitKeys,
final Configuration c) throws IOException {
return createTable(new HTableDescriptor(tableName), families, splitKeys, c);
}
@@ -1482,7 +1482,7 @@ public class HBaseTestingUtility extends HBaseCommonTestingUtility {
* @return An HTable instance for the created table.
* @throws IOException
*/
- public HTable createTable(TableName tableName, byte[] family, int numVersions)
+ public Table createTable(TableName tableName, byte[] family, int numVersions)
throws IOException {
return createTable(tableName, new byte[][]{family}, numVersions);
}
@@ -1495,7 +1495,7 @@ public class HBaseTestingUtility extends HBaseCommonTestingUtility {
* @return An HTable instance for the created table.
* @throws IOException
*/
- public HTable createTable(TableName tableName, byte[][] families, int numVersions)
+ public Table createTable(TableName tableName, byte[][] families, int numVersions)
throws IOException {
return createTable(tableName, families, numVersions, (byte[][]) null);
}
@@ -1509,18 +1509,18 @@ public class HBaseTestingUtility extends HBaseCommonTestingUtility {
* @return An HTable instance for the created table.
* @throws IOException
*/
- public HTable createTable(TableName tableName, byte[][] families, int numVersions,
+ public Table createTable(TableName tableName, byte[][] families, int numVersions,
byte[][] splitKeys) throws IOException {
HTableDescriptor desc = new HTableDescriptor(tableName);
for (byte[] family : families) {
HColumnDescriptor hcd = new HColumnDescriptor(family).setMaxVersions(numVersions);
desc.addFamily(hcd);
}
- getHBaseAdmin().createTable(desc, splitKeys);
+ getAdmin().createTable(desc, splitKeys);
// HBaseAdmin only waits for regions to appear in hbase:meta we should wait until they are
// assigned
waitUntilAllRegionsAssigned(tableName);
- return (HTable) getConnection().getTable(tableName);
+ return getConnection().getTable(tableName);
}
/**
@@ -1531,7 +1531,7 @@ public class HBaseTestingUtility extends HBaseCommonTestingUtility {
* @return An HTable instance for the created table.
* @throws IOException
*/
- public HTable createMultiRegionTable(TableName tableName, byte[][] families, int numVersions)
+ public Table createMultiRegionTable(TableName tableName, byte[][] families, int numVersions)
throws IOException {
return createTable(tableName, families, numVersions, KEYS_FOR_HBA_CREATE_TABLE);
}
@@ -1545,7 +1545,7 @@ public class HBaseTestingUtility extends HBaseCommonTestingUtility {
* @return An HTable instance for the created table.
* @throws IOException
*/
- public HTable createTable(TableName tableName, byte[][] families,
+ public Table createTable(TableName tableName, byte[][] families,
int numVersions, int blockSize) throws IOException {
HTableDescriptor desc = new HTableDescriptor(tableName);
for (byte[] family : families) {
@@ -1554,14 +1554,14 @@ public class HBaseTestingUtility extends HBaseCommonTestingUtility {
.setBlocksize(blockSize);
desc.addFamily(hcd);
}
- getHBaseAdmin().createTable(desc);
+ getAdmin().createTable(desc);
// HBaseAdmin only waits for regions to appear in hbase:meta we should wait until they are
// assigned
waitUntilAllRegionsAssigned(tableName);
- return (HTable) getConnection().getTable(tableName);
+ return getConnection().getTable(tableName);
}
- public HTable createTable(TableName tableName, byte[][] families,
+ public Table createTable(TableName tableName, byte[][] families,
int numVersions, int blockSize, String cpName) throws IOException {
HTableDescriptor desc = new HTableDescriptor(tableName);
for (byte[] family : families) {
@@ -1573,11 +1573,11 @@ public class HBaseTestingUtility extends HBaseCommonTestingUtility {
if(cpName != null) {
desc.addCoprocessor(cpName);
}
- getHBaseAdmin().createTable(desc);
+ getAdmin().createTable(desc);
// HBaseAdmin only waits for regions to appear in hbase:meta we should wait until they are
// assigned
waitUntilAllRegionsAssigned(tableName);
- return (HTable) getConnection().getTable(tableName);
+ return getConnection().getTable(tableName);
}
/**
@@ -1588,7 +1588,7 @@ public class HBaseTestingUtility extends HBaseCommonTestingUtility {
* @return An HTable instance for the created table.
* @throws IOException
*/
- public HTable createTable(TableName tableName, byte[][] families,
+ public Table createTable(TableName tableName, byte[][] families,
int[] numVersions)
throws IOException {
HTableDescriptor desc = new HTableDescriptor(tableName);
@@ -1599,11 +1599,11 @@ public class HBaseTestingUtility extends HBaseCommonTestingUtility {
desc.addFamily(hcd);
i++;
}
- getHBaseAdmin().createTable(desc);
+ getAdmin().createTable(desc);
// HBaseAdmin only waits for regions to appear in hbase:meta we should wait until they are
// assigned
waitUntilAllRegionsAssigned(tableName);
- return (HTable) getConnection().getTable(tableName);
+ return getConnection().getTable(tableName);
}
/**
@@ -1614,16 +1614,16 @@ public class HBaseTestingUtility extends HBaseCommonTestingUtility {
* @return An HTable instance for the created table.
* @throws IOException
*/
- public HTable createTable(TableName tableName, byte[] family, byte[][] splitRows)
+ public Table createTable(TableName tableName, byte[] family, byte[][] splitRows)
throws IOException {
HTableDescriptor desc = new HTableDescriptor(tableName);
HColumnDescriptor hcd = new HColumnDescriptor(family);
desc.addFamily(hcd);
- getHBaseAdmin().createTable(desc, splitRows);
+ getAdmin().createTable(desc, splitRows);
// HBaseAdmin only waits for regions to appear in hbase:meta we should wait until they are
// assigned
waitUntilAllRegionsAssigned(tableName);
- return (HTable) getConnection().getTable(tableName);
+ return getConnection().getTable(tableName);
}
/**
@@ -1633,7 +1633,7 @@ public class HBaseTestingUtility extends HBaseCommonTestingUtility {
* @return An HTable instance for the created table.
* @throws IOException
*/
- public HTable createMultiRegionTable(TableName tableName, byte[] family) throws IOException {
+ public Table createMultiRegionTable(TableName tableName, byte[] family) throws IOException {
return createTable(tableName, family, KEYS_FOR_HBA_CREATE_TABLE);
}
@@ -1683,11 +1683,11 @@ public class HBaseTestingUtility extends HBaseCommonTestingUtility {
*/
public void deleteTable(TableName tableName) throws IOException {
try {
- getHBaseAdmin().disableTable(tableName);
+ getAdmin().disableTable(tableName);
} catch (TableNotEnabledException e) {
LOG.debug("Table: " + tableName + " already disabled, so just deleting it.");
}
- getHBaseAdmin().deleteTable(tableName);
+ getAdmin().deleteTable(tableName);
}
/**
@@ -1873,8 +1873,8 @@ public class HBaseTestingUtility extends HBaseCommonTestingUtility {
* @return HTable to that new table
* @throws IOException
*/
- public HTable deleteTableData(TableName tableName) throws IOException {
- HTable table = (HTable) getConnection().getTable(tableName);
+ public Table deleteTableData(TableName tableName) throws IOException {
+ Table table = getConnection().getTable(tableName);
Scan scan = new Scan();
ResultScanner resScan = table.getScanner(scan);
for(Result res : resScan) {
@@ -1893,14 +1893,14 @@ public class HBaseTestingUtility extends HBaseCommonTestingUtility {
* @param preserveRegions keep the existing split points
* @return HTable for the new table
*/
- public HTable truncateTable(final TableName tableName, final boolean preserveRegions) throws
+ public Table truncateTable(final TableName tableName, final boolean preserveRegions) throws
IOException {
- Admin admin = getHBaseAdmin();
+ Admin admin = getAdmin();
if (!admin.isTableDisabled(tableName)) {
admin.disableTable(tableName);
}
admin.truncateTable(tableName, preserveRegions);
- return (HTable) getConnection().getTable(tableName);
+ return getConnection().getTable(tableName);
}
/**
@@ -1912,7 +1912,7 @@ public class HBaseTestingUtility extends HBaseCommonTestingUtility {
* @param tableName table which must exist.
* @return HTable for the new table
*/
- public HTable truncateTable(final TableName tableName) throws IOException {
+ public Table truncateTable(final TableName tableName) throws IOException {
return truncateTable(tableName, false);
}
@@ -2783,7 +2783,7 @@ public class HBaseTestingUtility extends HBaseCommonTestingUtility {
* @throws IOException
*/
public void closeRegion(byte[] regionName) throws IOException {
- getHBaseAdmin().closeRegion(regionName, null);
+ getAdmin().closeRegion(regionName, null);
}
/**
@@ -2819,7 +2819,7 @@ public class HBaseTestingUtility extends HBaseCommonTestingUtility {
public HRegion getSplittableRegion(TableName tableName, int maxAttempts) {
List regions = getHBaseCluster().getRegions(tableName);
int regCount = regions.size();
- Set attempted = new HashSet();
+ Set attempted = new HashSet<>();
int idx;
int attempts = 0;
do {
@@ -3719,7 +3719,7 @@ public class HBaseTestingUtility extends HBaseCommonTestingUtility {
final long timeout) throws IOException, InterruptedException {
long timeoutTime = System.currentTimeMillis() + timeout;
while (true) {
- List regions = getHBaseAdmin().getOnlineRegions(server);
+ List regions = getAdmin().getOnlineRegions(server);
if (regions.contains(hri)) return;
long now = System.currentTimeMillis();
if (now > timeoutTime) break;
@@ -3738,7 +3738,7 @@ public class HBaseTestingUtility extends HBaseCommonTestingUtility {
final long timeout) throws IOException, InterruptedException {
long timeoutTime = System.currentTimeMillis() + timeout;
while (true) {
- List regions = getHBaseAdmin().getOnlineRegions(server);
+ List regions = getAdmin().getOnlineRegions(server);
if (regions.contains(hri)) {
List rsThreads =
getHBaseCluster().getLiveRegionServerThreads();
@@ -3836,7 +3836,7 @@ public class HBaseTestingUtility extends HBaseCommonTestingUtility {
@Override
public boolean evaluate() throws IOException {
- return getHBaseAdmin().tableExists(tableName) && getHBaseAdmin().isTableEnabled(tableName);
+ return getAdmin().tableExists(tableName) && getAdmin().isTableEnabled(tableName);
}
};
}
@@ -3853,7 +3853,7 @@ public class HBaseTestingUtility extends HBaseCommonTestingUtility {
@Override
public boolean evaluate() throws IOException {
- return getHBaseAdmin().isTableDisabled(tableName);
+ return getAdmin().isTableDisabled(tableName);
}
};
}
@@ -3870,10 +3870,10 @@ public class HBaseTestingUtility extends HBaseCommonTestingUtility {
@Override
public boolean evaluate() throws IOException {
- boolean tableAvailable = getHBaseAdmin().isTableAvailable(tableName);
+ boolean tableAvailable = getAdmin().isTableAvailable(tableName);
if (tableAvailable) {
try {
- Canary.sniff(getHBaseAdmin(), tableName);
+ Canary.sniff(getAdmin(), tableName);
} catch (Exception e) {
throw new IOException("Canary sniff failed for table " + tableName, e);
}
@@ -3947,7 +3947,7 @@ public class HBaseTestingUtility extends HBaseCommonTestingUtility {
* @return the list of column descriptors
*/
public static List generateColumnDescriptors(final String prefix) {
- List htds = new ArrayList();
+ List htds = new ArrayList<>();
long familyId = 0;
for (Compression.Algorithm compressionType: getSupportedCompressionAlgorithms()) {
for (DataBlockEncoding encodingType: DataBlockEncoding.values()) {
@@ -3971,7 +3971,7 @@ public class HBaseTestingUtility extends HBaseCommonTestingUtility {
*/
public static Compression.Algorithm[] getSupportedCompressionAlgorithms() {
String[] allAlgos = HFile.getSupportedCompressionAlgorithms();
- List supportedAlgos = new ArrayList();
+ List supportedAlgos = new ArrayList<>();
for (String algoName : allAlgos) {
try {
Compression.Algorithm algo = Compression.getCompressionAlgorithmByName(algoName);
@@ -3991,7 +3991,7 @@ public class HBaseTestingUtility extends HBaseCommonTestingUtility {
scan.setReversed(true);
scan.addFamily(family);
try (RegionScanner scanner = r.getScanner(scan)) {
- List cells = new ArrayList(1);
+ List cells = new ArrayList<>(1);
scanner.next(cells);
if (r.getRegionInfo().isMetaRegion() && !isTargetTable(row, cells.get(0))) {
return null;
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/HConnectionTestingUtility.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/HConnectionTestingUtility.java
index f0da437..c8ccd2a 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/HConnectionTestingUtility.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/HConnectionTestingUtility.java
@@ -126,7 +126,7 @@ public class HConnectionTestingUtility {
RpcRetryingCallerFactory.instantiate(conf,
RetryingCallerInterceptorFactory.NO_OP_INTERCEPTOR, null));
Mockito.when(c.getRpcControllerFactory()).thenReturn(Mockito.mock(RpcControllerFactory.class));
- HTableInterface t = Mockito.mock(HTableInterface.class);
+ Table t = Mockito.mock(Table.class);
Mockito.when(c.getTable((TableName)Mockito.any())).thenReturn(t);
ResultScanner rs = Mockito.mock(ResultScanner.class);
Mockito.when(t.getScanner((Scan)Mockito.any())).thenReturn(rs);
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestBlockEvictionFromClient.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestBlockEvictionFromClient.java
index f94ed2f..ba75d6e 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestBlockEvictionFromClient.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestBlockEvictionFromClient.java
@@ -165,7 +165,7 @@ public class TestBlockEvictionFromClient {
@Test
public void testBlockEvictionWithParallelScans() throws Exception {
- HTable table = null;
+ Table table = null;
try {
latch = new CountDownLatch(1);
TableName tableName = TableName.valueOf("testBlockEvictionWithParallelScans");
@@ -173,7 +173,7 @@ public class TestBlockEvictionFromClient {
table = TEST_UTIL.createTable(tableName, FAMILIES_1, 1, 1024,
CustomInnerRegionObserver.class.getName());
// get the block cache and region
- RegionLocator locator = table.getRegionLocator();
+ RegionLocator locator = TEST_UTIL.getConnection().getRegionLocator(tableName);
String regionName = locator.getAllRegionLocations().get(0).getRegionInfo().getEncodedName();
Region region = TEST_UTIL.getRSForFirstRegionInTable(tableName).getFromOnlineRegions(
regionName);
@@ -253,7 +253,7 @@ public class TestBlockEvictionFromClient {
@Test
public void testParallelGetsAndScans() throws IOException, InterruptedException {
- HTable table = null;
+ Table table = null;
try {
latch = new CountDownLatch(2);
// Check if get() returns blocks on its close() itself
@@ -264,7 +264,7 @@ public class TestBlockEvictionFromClient {
table = TEST_UTIL.createTable(tableName, FAMILIES_1, 1, 1024,
CustomInnerRegionObserver.class.getName());
// get the block cache and region
- RegionLocator locator = table.getRegionLocator();
+ RegionLocator locator = TEST_UTIL.getConnection().getRegionLocator(tableName);
String regionName = locator.getAllRegionLocations().get(0).getRegionInfo().getEncodedName();
Region region = TEST_UTIL.getRSForFirstRegionInTable(tableName).getFromOnlineRegions(
regionName);
@@ -312,7 +312,7 @@ public class TestBlockEvictionFromClient {
@Test
public void testGetWithCellsInDifferentFiles() throws IOException, InterruptedException {
- HTable table = null;
+ Table table = null;
try {
latch = new CountDownLatch(1);
// Check if get() returns blocks on its close() itself
@@ -323,7 +323,7 @@ public class TestBlockEvictionFromClient {
table = TEST_UTIL.createTable(tableName, FAMILIES_1, 1, 1024,
CustomInnerRegionObserver.class.getName());
// get the block cache and region
- RegionLocator locator = table.getRegionLocator();
+ RegionLocator locator = TEST_UTIL.getConnection().getRegionLocator(tableName);
String regionName = locator.getAllRegionLocations().get(0).getRegionInfo().getEncodedName();
Region region = TEST_UTIL.getRSForFirstRegionInTable(tableName).getFromOnlineRegions(
regionName);
@@ -374,7 +374,7 @@ public class TestBlockEvictionFromClient {
// TODO : check how block index works here
public void testGetsWithMultiColumnsAndExplicitTracker()
throws IOException, InterruptedException {
- HTable table = null;
+ Table table = null;
try {
latch = new CountDownLatch(1);
// Check if get() returns blocks on its close() itself
@@ -385,7 +385,7 @@ public class TestBlockEvictionFromClient {
table = TEST_UTIL.createTable(tableName, FAMILIES_1, 1, 1024,
CustomInnerRegionObserver.class.getName());
// get the block cache and region
- RegionLocator locator = table.getRegionLocator();
+ RegionLocator locator = TEST_UTIL.getConnection().getRegionLocator(tableName);
String regionName = locator.getAllRegionLocations().get(0).getRegionInfo().getEncodedName();
Region region = TEST_UTIL.getRSForFirstRegionInTable(tableName).getFromOnlineRegions(
regionName);
@@ -462,7 +462,7 @@ public class TestBlockEvictionFromClient {
@Test
public void testGetWithMultipleColumnFamilies() throws IOException, InterruptedException {
- HTable table = null;
+ Table table = null;
try {
latch = new CountDownLatch(1);
// Check if get() returns blocks on its close() itself
@@ -478,7 +478,7 @@ public class TestBlockEvictionFromClient {
table = TEST_UTIL.createTable(tableName, fams, 1, 1024,
CustomInnerRegionObserver.class.getName());
// get the block cache and region
- RegionLocator locator = table.getRegionLocator();
+ RegionLocator locator = TEST_UTIL.getConnection().getRegionLocator(tableName);
String regionName = locator.getAllRegionLocations().get(0).getRegionInfo().getEncodedName();
Region region = TEST_UTIL.getRSForFirstRegionInTable(tableName).getFromOnlineRegions(
regionName);
@@ -557,12 +557,12 @@ public class TestBlockEvictionFromClient {
@Test
public void testBlockRefCountAfterSplits() throws IOException, InterruptedException {
- HTable table = null;
+ Table table = null;
try {
TableName tableName = TableName.valueOf("testBlockRefCountAfterSplits");
table = TEST_UTIL.createTable(tableName, FAMILIES_1, 1, 1024);
// get the block cache and region
- RegionLocator locator = table.getRegionLocator();
+ RegionLocator locator = TEST_UTIL.getConnection().getRegionLocator(tableName);
String regionName = locator.getAllRegionLocations().get(0).getRegionInfo().getEncodedName();
Region region =
TEST_UTIL.getRSForFirstRegionInTable(tableName).getFromOnlineRegions(regionName);
@@ -608,7 +608,7 @@ public class TestBlockEvictionFromClient {
@Test
public void testMultiGets() throws IOException, InterruptedException {
- HTable table = null;
+ Table table = null;
try {
latch = new CountDownLatch(2);
// Check if get() returns blocks on its close() itself
@@ -619,7 +619,7 @@ public class TestBlockEvictionFromClient {
table = TEST_UTIL.createTable(tableName, FAMILIES_1, 1, 1024,
CustomInnerRegionObserver.class.getName());
// get the block cache and region
- RegionLocator locator = table.getRegionLocator();
+ RegionLocator locator = TEST_UTIL.getConnection().getRegionLocator(tableName);
String regionName = locator.getAllRegionLocations().get(0).getRegionInfo().getEncodedName();
Region region = TEST_UTIL.getRSForFirstRegionInTable(tableName).getFromOnlineRegions(
regionName);
@@ -687,7 +687,7 @@ public class TestBlockEvictionFromClient {
}
@Test
public void testScanWithMultipleColumnFamilies() throws IOException, InterruptedException {
- HTable table = null;
+ Table table = null;
try {
latch = new CountDownLatch(1);
// Check if get() returns blocks on its close() itself
@@ -702,7 +702,7 @@ public class TestBlockEvictionFromClient {
table = TEST_UTIL.createTable(tableName, fams, 1, 1024,
CustomInnerRegionObserver.class.getName());
// get the block cache and region
- RegionLocator locator = table.getRegionLocator();
+ RegionLocator locator = TEST_UTIL.getConnection().getRegionLocator(tableName);
String regionName = locator.getAllRegionLocations().get(0).getRegionInfo().getEncodedName();
Region region = TEST_UTIL.getRSForFirstRegionInTable(tableName).getFromOnlineRegions(
regionName);
@@ -791,7 +791,7 @@ public class TestBlockEvictionFromClient {
@Test
public void testParallelGetsAndScanWithWrappedRegionScanner() throws IOException,
InterruptedException {
- HTable table = null;
+ Table table = null;
try {
latch = new CountDownLatch(2);
// Check if get() returns blocks on its close() itself
@@ -802,7 +802,7 @@ public class TestBlockEvictionFromClient {
table = TEST_UTIL.createTable(tableName, FAMILIES_1, 1, 1024,
CustomInnerRegionObserverWrapper.class.getName());
// get the block cache and region
- RegionLocator locator = table.getRegionLocator();
+ RegionLocator locator = TEST_UTIL.getConnection().getRegionLocator(tableName);
String regionName = locator.getAllRegionLocations().get(0).getRegionInfo().getEncodedName();
Region region = TEST_UTIL.getRSForFirstRegionInTable(tableName).getFromOnlineRegions(
regionName);
@@ -859,7 +859,7 @@ public class TestBlockEvictionFromClient {
private void testScanWithCompactionInternals(String tableNameStr, boolean reversed)
throws IOException, InterruptedException {
- HTable table = null;
+ Table table = null;
try {
latch = new CountDownLatch(1);
compactionLatch = new CountDownLatch(1);
@@ -868,7 +868,7 @@ public class TestBlockEvictionFromClient {
table = TEST_UTIL.createTable(tableName, FAMILIES_1, 1, 1024,
CustomInnerRegionObserverWrapper.class.getName());
// get the block cache and region
- RegionLocator locator = table.getRegionLocator();
+ RegionLocator locator = TEST_UTIL.getConnection().getRegionLocator(tableName);
String regionName = locator.getAllRegionLocations().get(0).getRegionInfo().getEncodedName();
Region region = TEST_UTIL.getRSForFirstRegionInTable(tableName).getFromOnlineRegions(
regionName);
@@ -977,7 +977,7 @@ public class TestBlockEvictionFromClient {
public void testBlockEvictionAfterHBASE13082WithCompactionAndFlush()
throws IOException, InterruptedException {
// do flush and scan in parallel
- HTable table = null;
+ Table table = null;
try {
latch = new CountDownLatch(1);
compactionLatch = new CountDownLatch(1);
@@ -987,7 +987,7 @@ public class TestBlockEvictionFromClient {
table = TEST_UTIL.createTable(tableName, FAMILIES_1, 1, 1024,
CustomInnerRegionObserverWrapper.class.getName());
// get the block cache and region
- RegionLocator locator = table.getRegionLocator();
+ RegionLocator locator = TEST_UTIL.getConnection().getRegionLocator(tableName);
String regionName = locator.getAllRegionLocations().get(0).getRegionInfo().getEncodedName();
Region region = TEST_UTIL.getRSForFirstRegionInTable(tableName).getFromOnlineRegions(
regionName);
@@ -1107,7 +1107,7 @@ public class TestBlockEvictionFromClient {
@Test
public void testScanWithException() throws IOException, InterruptedException {
- HTable table = null;
+ Table table = null;
try {
latch = new CountDownLatch(1);
exceptionLatch = new CountDownLatch(1);
@@ -1117,7 +1117,7 @@ public class TestBlockEvictionFromClient {
table = TEST_UTIL.createTable(tableName, FAMILIES_1, 1, 1024,
CustomInnerRegionObserverWrapper.class.getName());
// get the block cache and region
- RegionLocator locator = table.getRegionLocator();
+ RegionLocator locator = TEST_UTIL.getConnection().getRegionLocator(tableName);
String regionName = locator.getAllRegionLocations().get(0).getRegionInfo().getEncodedName();
Region region = TEST_UTIL.getRSForFirstRegionInTable(tableName).getFromOnlineRegions(
regionName);
@@ -1225,7 +1225,7 @@ public class TestBlockEvictionFromClient {
}
}
- private void insertData(HTable table) throws IOException {
+ private void insertData(Table table) throws IOException {
Put put = new Put(ROW);
put.addColumn(FAMILY, QUALIFIER, data);
table.put(put);
@@ -1238,7 +1238,7 @@ public class TestBlockEvictionFromClient {
table.put(put);
}
- private ScanThread[] initiateScan(HTable table, boolean reverse) throws IOException,
+ private ScanThread[] initiateScan(Table table, boolean reverse) throws IOException,
InterruptedException {
ScanThread[] scanThreads = new ScanThread[NO_OF_THREADS];
for (int i = 0; i < NO_OF_THREADS; i++) {
@@ -1250,7 +1250,7 @@ public class TestBlockEvictionFromClient {
return scanThreads;
}
- private GetThread[] initiateGet(HTable table, boolean tracker, boolean multipleCFs)
+ private GetThread[] initiateGet(Table table, boolean tracker, boolean multipleCFs)
throws IOException, InterruptedException {
GetThread[] getThreads = new GetThread[NO_OF_THREADS];
for (int i = 0; i < NO_OF_THREADS; i++) {
@@ -1262,7 +1262,7 @@ public class TestBlockEvictionFromClient {
return getThreads;
}
- private MultiGetThread[] initiateMultiGet(HTable table)
+ private MultiGetThread[] initiateMultiGet(Table table)
throws IOException, InterruptedException {
MultiGetThread[] multiGetThreads = new MultiGetThread[NO_OF_THREADS];
for (int i = 0; i < NO_OF_THREADS; i++) {
@@ -1337,9 +1337,9 @@ public class TestBlockEvictionFromClient {
}
private static class MultiGetThread extends Thread {
- private final HTable table;
+ private final Table table;
private final List gets = new ArrayList();
- public MultiGetThread(HTable table) {
+ public MultiGetThread(Table table) {
this.table = table;
}
@Override
@@ -1357,11 +1357,11 @@ public class TestBlockEvictionFromClient {
}
private static class GetThread extends Thread {
- private final HTable table;
+ private final Table table;
private final boolean tracker;
private final boolean multipleCFs;
- public GetThread(HTable table, boolean tracker, boolean multipleCFs) {
+ public GetThread(Table table, boolean tracker, boolean multipleCFs) {
this.table = table;
this.tracker = tracker;
this.multipleCFs = multipleCFs;
@@ -1376,7 +1376,7 @@ public class TestBlockEvictionFromClient {
}
}
- private void initiateGet(HTable table) throws IOException {
+ private void initiateGet(Table table) throws IOException {
Get get = new Get(ROW);
if (tracker) {
// Change this
@@ -1421,10 +1421,10 @@ public class TestBlockEvictionFromClient {
}
private static class ScanThread extends Thread {
- private final HTable table;
+ private final Table table;
private final boolean reverse;
- public ScanThread(HTable table, boolean reverse) {
+ public ScanThread(Table table, boolean reverse) {
this.table = table;
this.reverse = reverse;
}
@@ -1438,7 +1438,7 @@ public class TestBlockEvictionFromClient {
}
}
- private void initiateScan(HTable table) throws IOException {
+ private void initiateScan(Table table) throws IOException {
Scan scan = new Scan();
if (reverse) {
scan.setReversed(true);
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestClientPushback.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestClientPushback.java
index e82b1c7..baec37e 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestClientPushback.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestClientPushback.java
@@ -61,7 +61,7 @@ public class TestClientPushback {
private static final TableName tableName = TableName.valueOf("client-pushback");
private static final byte[] family = Bytes.toBytes("f");
private static final byte[] qualifier = Bytes.toBytes("q");
- private static long flushSizeBytes = 1024;
+ private static final long flushSizeBytes = 1024;
@BeforeClass
public static void setupCluster() throws Exception{
@@ -91,7 +91,7 @@ public class TestClientPushback {
Configuration conf = UTIL.getConfiguration();
ClusterConnection conn = (ClusterConnection) ConnectionFactory.createConnection(conf);
- HTable table = (HTable) conn.getTable(tableName);
+ Table table = conn.getTable(tableName);
HRegionServer rs = UTIL.getHBaseCluster().getRegionServer(0);
Region region = rs.getOnlineRegions(tableName).get(0);
@@ -130,13 +130,13 @@ public class TestClientPushback {
// Reach into the connection and submit work directly to AsyncProcess so we can
// monitor how long the submission was delayed via a callback
- List ops = new ArrayList(1);
+ List ops = new ArrayList<>(1);
ops.add(p);
final CountDownLatch latch = new CountDownLatch(1);
final AtomicLong endTime = new AtomicLong();
long startTime = EnvironmentEdgeManager.currentTime();
- table.mutator.ap.submit(tableName, ops, true, new Batch.Callback() {
+ ((HTable) table).mutator.ap.submit(tableName, ops, true, new Batch.Callback() {
@Override
public void update(byte[] region, byte[] row, Result result) {
endTime.set(EnvironmentEdgeManager.currentTime());
@@ -172,7 +172,7 @@ public class TestClientPushback {
public void testMutateRowStats() throws IOException {
Configuration conf = UTIL.getConfiguration();
ClusterConnection conn = (ClusterConnection) ConnectionFactory.createConnection(conf);
- HTable table = (HTable) conn.getTable(tableName);
+ Table table = conn.getTable(tableName);
HRegionServer rs = UTIL.getHBaseCluster().getRegionServer(0);
Region region = rs.getOnlineRegions(tableName).get(0);
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 1ddb13b..ca4b609 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
@@ -3809,51 +3809,6 @@ public class TestFromClientSide {
}
@Test
- public void testRowsPutBufferedOneFlush() throws IOException {
- final byte [] CONTENTS_FAMILY = Bytes.toBytes("contents");
- final byte [] SMALL_FAMILY = Bytes.toBytes("smallfam");
- final byte [] value = Bytes.toBytes("abcd");
- final int NB_BATCH_ROWS = 10;
- Table t = TEST_UTIL.createTable(TableName.valueOf("testRowsPutBufferedOneFlush"),
- new byte[][] { CONTENTS_FAMILY, SMALL_FAMILY });
-
- // Only do this test if it is a HTable
- if(t instanceof HTableInterface) {
- HTable table = (HTable) t;
- table.setAutoFlushTo(false);
- ArrayList rowsUpdate = new ArrayList();
- for (int i = 0; i < NB_BATCH_ROWS * 10; i++) {
- byte[] row = Bytes.toBytes("row" + i);
- Put put = new Put(row);
- put.setDurability(Durability.SKIP_WAL);
- put.addColumn(CONTENTS_FAMILY, null, value);
- rowsUpdate.add(put);
- }
- table.put(rowsUpdate);
-
- Scan scan = new Scan();
- scan.addFamily(CONTENTS_FAMILY);
- ResultScanner scanner = table.getScanner(scan);
- int nbRows = 0;
- for (@SuppressWarnings("unused") Result row : scanner)
- nbRows++;
- assertEquals(0, nbRows);
- scanner.close();
-
- table.flushCommits();
-
- scan = new Scan();
- scan.addFamily(CONTENTS_FAMILY);
- scanner = table.getScanner(scan);
- nbRows = 0;
- for (@SuppressWarnings("unused") Result row : scanner)
- nbRows++;
- assertEquals(NB_BATCH_ROWS * 10, nbRows);
- table.close();
- }
- }
-
- @Test
public void testRowsPutBufferedManyManyFlushes() throws IOException {
final byte[] CONTENTS_FAMILY = Bytes.toBytes("contents");
final byte[] SMALL_FAMILY = Bytes.toBytes("smallfam");
@@ -4175,92 +4130,89 @@ public class TestFromClientSide {
final byte[] beforeThirdRow = Bytes.toBytes("row33");
final byte[] beforeForthRow = Bytes.toBytes("row44");
- try (Table t =
+ try (Table table =
TEST_UTIL.createTable(tableName,
new byte[][] { HConstants.CATALOG_FAMILY, Bytes.toBytes("info2") }, 1, 1024);
- RegionLocator locator = TEST_UTIL.getConnection().getRegionLocator(tableName)) {
- if (t instanceof HTableInterface) {
- HTableInterface table = (HTableInterface) t;
-
- // set block size to 64 to making 2 kvs into one block, bypassing the walkForwardInSingleRow
- // in Store.rowAtOrBeforeFromStoreFile
- String regionName = locator.getAllRegionLocations().get(0).getRegionInfo().getEncodedName();
- Region region =
- TEST_UTIL.getRSForFirstRegionInTable(tableName).getFromOnlineRegions(regionName);
- Put put1 = new Put(firstRow);
- Put put2 = new Put(secondRow);
- Put put3 = new Put(thirdRow);
- Put put4 = new Put(forthRow);
- byte[] one = new byte[] { 1 };
- byte[] two = new byte[] { 2 };
- byte[] three = new byte[] { 3 };
- byte[] four = new byte[] { 4 };
-
- put1.addColumn(HConstants.CATALOG_FAMILY, null, one);
- put2.addColumn(HConstants.CATALOG_FAMILY, null, two);
- put3.addColumn(HConstants.CATALOG_FAMILY, null, three);
- put4.addColumn(HConstants.CATALOG_FAMILY, null, four);
- table.put(put1);
- table.put(put2);
- table.put(put3);
- table.put(put4);
- region.flush(true);
-
- Result result;
-
- // Test before first that null is returned
- result = getReverseScanResult(table, beforeFirstRow,
- HConstants.CATALOG_FAMILY);
- assertNull(result);
-
- // Test at first that first is returned
- result = getReverseScanResult(table, firstRow, HConstants.CATALOG_FAMILY);
- assertTrue(result.containsColumn(HConstants.CATALOG_FAMILY, null));
- assertTrue(Bytes.equals(result.getRow(), firstRow));
- assertTrue(Bytes.equals(result.getValue(HConstants.CATALOG_FAMILY, null), one));
-
- // Test in between first and second that first is returned
- result = getReverseScanResult(table, beforeSecondRow, HConstants.CATALOG_FAMILY);
- assertTrue(result.containsColumn(HConstants.CATALOG_FAMILY, null));
- assertTrue(Bytes.equals(result.getRow(), firstRow));
- assertTrue(Bytes.equals(result.getValue(HConstants.CATALOG_FAMILY, null), one));
-
- // Test at second make sure second is returned
- result = getReverseScanResult(table, secondRow, HConstants.CATALOG_FAMILY);
- assertTrue(result.containsColumn(HConstants.CATALOG_FAMILY, null));
- assertTrue(Bytes.equals(result.getRow(), secondRow));
- assertTrue(Bytes.equals(result.getValue(HConstants.CATALOG_FAMILY, null), two));
-
- // Test in second and third, make sure second is returned
- result = getReverseScanResult(table, beforeThirdRow, HConstants.CATALOG_FAMILY);
- assertTrue(result.containsColumn(HConstants.CATALOG_FAMILY, null));
- assertTrue(Bytes.equals(result.getRow(), secondRow));
- assertTrue(Bytes.equals(result.getValue(HConstants.CATALOG_FAMILY, null), two));
-
- // Test at third make sure third is returned
- result = getReverseScanResult(table, thirdRow, HConstants.CATALOG_FAMILY);
- assertTrue(result.containsColumn(HConstants.CATALOG_FAMILY, null));
- assertTrue(Bytes.equals(result.getRow(), thirdRow));
- assertTrue(Bytes.equals(result.getValue(HConstants.CATALOG_FAMILY, null), three));
-
- // Test in third and forth, make sure third is returned
- result = getReverseScanResult(table, beforeForthRow, HConstants.CATALOG_FAMILY);
- assertTrue(result.containsColumn(HConstants.CATALOG_FAMILY, null));
- assertTrue(Bytes.equals(result.getRow(), thirdRow));
- assertTrue(Bytes.equals(result.getValue(HConstants.CATALOG_FAMILY, null), three));
-
- // Test at forth make sure forth is returned
- result = getReverseScanResult(table, forthRow, HConstants.CATALOG_FAMILY);
- assertTrue(result.containsColumn(HConstants.CATALOG_FAMILY, null));
- assertTrue(Bytes.equals(result.getRow(), forthRow));
- assertTrue(Bytes.equals(result.getValue(HConstants.CATALOG_FAMILY, null), four));
-
- // Test after forth make sure forth is returned
- result = getReverseScanResult(table, Bytes.add(forthRow, one), HConstants.CATALOG_FAMILY);
- assertTrue(result.containsColumn(HConstants.CATALOG_FAMILY, null));
- assertTrue(Bytes.equals(result.getRow(), forthRow));
- assertTrue(Bytes.equals(result.getValue(HConstants.CATALOG_FAMILY, null), four));
- }
+ RegionLocator locator = TEST_UTIL.getConnection().getRegionLocator(tableName)) {
+
+ // set block size to 64 to making 2 kvs into one block, bypassing the walkForwardInSingleRow
+ // in Store.rowAtOrBeforeFromStoreFile
+ String regionName = locator.getAllRegionLocations().get(0).getRegionInfo().getEncodedName();
+ Region region =
+ TEST_UTIL.getRSForFirstRegionInTable(tableName).getFromOnlineRegions(regionName);
+ Put put1 = new Put(firstRow);
+ Put put2 = new Put(secondRow);
+ Put put3 = new Put(thirdRow);
+ Put put4 = new Put(forthRow);
+ byte[] one = new byte[] { 1 };
+ byte[] two = new byte[] { 2 };
+ byte[] three = new byte[] { 3 };
+ byte[] four = new byte[] { 4 };
+
+ put1.addColumn(HConstants.CATALOG_FAMILY, null, one);
+ put2.addColumn(HConstants.CATALOG_FAMILY, null, two);
+ put3.addColumn(HConstants.CATALOG_FAMILY, null, three);
+ put4.addColumn(HConstants.CATALOG_FAMILY, null, four);
+ table.put(put1);
+ table.put(put2);
+ table.put(put3);
+ table.put(put4);
+ region.flush(true);
+
+ Result result;
+
+ // Test before first that null is returned
+ result = getReverseScanResult(table, beforeFirstRow,
+ HConstants.CATALOG_FAMILY);
+ assertNull(result);
+
+ // Test at first that first is returned
+ result = getReverseScanResult(table, firstRow, HConstants.CATALOG_FAMILY);
+ assertTrue(result.containsColumn(HConstants.CATALOG_FAMILY, null));
+ assertTrue(Bytes.equals(result.getRow(), firstRow));
+ assertTrue(Bytes.equals(result.getValue(HConstants.CATALOG_FAMILY, null), one));
+
+ // Test in between first and second that first is returned
+ result = getReverseScanResult(table, beforeSecondRow, HConstants.CATALOG_FAMILY);
+ assertTrue(result.containsColumn(HConstants.CATALOG_FAMILY, null));
+ assertTrue(Bytes.equals(result.getRow(), firstRow));
+ assertTrue(Bytes.equals(result.getValue(HConstants.CATALOG_FAMILY, null), one));
+
+ // Test at second make sure second is returned
+ result = getReverseScanResult(table, secondRow, HConstants.CATALOG_FAMILY);
+ assertTrue(result.containsColumn(HConstants.CATALOG_FAMILY, null));
+ assertTrue(Bytes.equals(result.getRow(), secondRow));
+ assertTrue(Bytes.equals(result.getValue(HConstants.CATALOG_FAMILY, null), two));
+
+ // Test in second and third, make sure second is returned
+ result = getReverseScanResult(table, beforeThirdRow, HConstants.CATALOG_FAMILY);
+ assertTrue(result.containsColumn(HConstants.CATALOG_FAMILY, null));
+ assertTrue(Bytes.equals(result.getRow(), secondRow));
+ assertTrue(Bytes.equals(result.getValue(HConstants.CATALOG_FAMILY, null), two));
+
+ // Test at third make sure third is returned
+ result = getReverseScanResult(table, thirdRow, HConstants.CATALOG_FAMILY);
+ assertTrue(result.containsColumn(HConstants.CATALOG_FAMILY, null));
+ assertTrue(Bytes.equals(result.getRow(), thirdRow));
+ assertTrue(Bytes.equals(result.getValue(HConstants.CATALOG_FAMILY, null), three));
+
+ // Test in third and forth, make sure third is returned
+ result = getReverseScanResult(table, beforeForthRow, HConstants.CATALOG_FAMILY);
+ assertTrue(result.containsColumn(HConstants.CATALOG_FAMILY, null));
+ assertTrue(Bytes.equals(result.getRow(), thirdRow));
+ assertTrue(Bytes.equals(result.getValue(HConstants.CATALOG_FAMILY, null), three));
+
+ // Test at forth make sure forth is returned
+ result = getReverseScanResult(table, forthRow, HConstants.CATALOG_FAMILY);
+ assertTrue(result.containsColumn(HConstants.CATALOG_FAMILY, null));
+ assertTrue(Bytes.equals(result.getRow(), forthRow));
+ assertTrue(Bytes.equals(result.getValue(HConstants.CATALOG_FAMILY, null), four));
+
+ // Test after forth make sure forth is returned
+ result = getReverseScanResult(table, Bytes.add(forthRow, one), HConstants.CATALOG_FAMILY);
+ assertTrue(result.containsColumn(HConstants.CATALOG_FAMILY, null));
+ assertTrue(Bytes.equals(result.getRow(), forthRow));
+ assertTrue(Bytes.equals(result.getValue(HConstants.CATALOG_FAMILY, null), four));
}
}
@@ -5015,57 +4967,53 @@ public class TestFromClientSide {
TableName TABLE = TableName.valueOf("testGetRegionsInRange");
Table t = TEST_UTIL.createMultiRegionTable(TABLE, new byte[][] { FAMILY }, 10);
- if (t instanceof HTable){
- HTable table = (HTable) t;
-
- int numOfRegions = -1;
- try (RegionLocator r = table.getRegionLocator()) {
- numOfRegions = r.getStartKeys().length;
- }
- assertEquals(26, numOfRegions);
-
- // Get the regions in this range
- List regionsList = getRegionsInRange(TABLE, startKey, endKey);
- assertEquals(10, regionsList.size());
-
- // Change the start key
- startKey = Bytes.toBytes("fff");
- regionsList = getRegionsInRange(TABLE, startKey, endKey);
- assertEquals(7, regionsList.size());
-
- // Change the end key
- endKey = Bytes.toBytes("nnn");
- regionsList = getRegionsInRange(TABLE, startKey, endKey);
- assertEquals(8, regionsList.size());
-
- // Empty start key
- regionsList = getRegionsInRange(TABLE, HConstants.EMPTY_START_ROW, endKey);
- assertEquals(13, regionsList.size());
-
- // Empty end key
- regionsList = getRegionsInRange(TABLE, startKey, HConstants.EMPTY_END_ROW);
- assertEquals(21, regionsList.size());
-
- // Both start and end keys empty
- regionsList = getRegionsInRange(TABLE, HConstants.EMPTY_START_ROW,
- HConstants.EMPTY_END_ROW);
- assertEquals(26, regionsList.size());
-
- // Change the end key to somewhere in the last block
- endKey = Bytes.toBytes("zzz1");
- regionsList = getRegionsInRange(TABLE, startKey, endKey);
- assertEquals(21, regionsList.size());
-
- // Change the start key to somewhere in the first block
- startKey = Bytes.toBytes("aac");
- regionsList = getRegionsInRange(TABLE, startKey, endKey);
- assertEquals(26, regionsList.size());
-
- // Make start and end key the same
- startKey = endKey = Bytes.toBytes("ccc");
- regionsList = getRegionsInRange(TABLE, startKey, endKey);
- assertEquals(1, regionsList.size());
+ int numOfRegions = -1;
+ try (RegionLocator r = TEST_UTIL.getConnection().getRegionLocator(TABLE)) {
+ numOfRegions = r.getStartKeys().length;
}
+ assertEquals(26, numOfRegions);
+
+ // Get the regions in this range
+ List regionsList = getRegionsInRange(TABLE, startKey, endKey);
+ assertEquals(10, regionsList.size());
+
+ // Change the start key
+ startKey = Bytes.toBytes("fff");
+ regionsList = getRegionsInRange(TABLE, startKey, endKey);
+ assertEquals(7, regionsList.size());
+
+ // Change the end key
+ endKey = Bytes.toBytes("nnn");
+ regionsList = getRegionsInRange(TABLE, startKey, endKey);
+ assertEquals(8, regionsList.size());
+
+ // Empty start key
+ regionsList = getRegionsInRange(TABLE, HConstants.EMPTY_START_ROW, endKey);
+ assertEquals(13, regionsList.size());
+
+ // Empty end key
+ regionsList = getRegionsInRange(TABLE, startKey, HConstants.EMPTY_END_ROW);
+ assertEquals(21, regionsList.size());
+
+ // Both start and end keys empty
+ regionsList = getRegionsInRange(TABLE, HConstants.EMPTY_START_ROW,
+ HConstants.EMPTY_END_ROW);
+ assertEquals(26, regionsList.size());
+
+ // Change the end key to somewhere in the last block
+ endKey = Bytes.toBytes("zzz1");
+ regionsList = getRegionsInRange(TABLE, startKey, endKey);
+ assertEquals(21, regionsList.size());
+
+ // Change the start key to somewhere in the first block
+ startKey = Bytes.toBytes("aac");
+ regionsList = getRegionsInRange(TABLE, startKey, endKey);
+ assertEquals(26, regionsList.size());
+
+ // Make start and end key the same
+ startKey = endKey = Bytes.toBytes("ccc");
+ regionsList = getRegionsInRange(TABLE, startKey, endKey);
+ assertEquals(1, regionsList.size());
}
private List getRegionsInRange(TableName tableName, byte[] startKey,
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestFromClientSideNoCodec.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestFromClientSideNoCodec.java
index 550a4c9..3da8454 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestFromClientSideNoCodec.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestFromClientSideNoCodec.java
@@ -82,15 +82,12 @@ public class TestFromClientSideNoCodec {
Bytes.equals(cell.getValueArray(), cell.getValueOffset(), cell.getValueLength(),
f, 0, f.length));
}
- if(ht instanceof HTableInterface) {
- HTableInterface hti = (HTableInterface) ht;
- // Check getRowOrBefore
- byte[] f = fs[0];
- Get get = new Get(row);
- get.addFamily(f);
- r = ht.get(get);
- assertTrue(r.toString(), r.containsColumn(f, f));
- }
+ // Check getRowOrBefore
+ byte[] f = fs[0];
+ Get get = new Get(row);
+ get.addFamily(f);
+ r = ht.get(get);
+ assertTrue(r.toString(), r.containsColumn(f, f));
// Check scan.
ResultScanner scanner = ht.getScanner(new Scan());
int count = 0;
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 190a1d2..4723fa8 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
@@ -176,45 +176,56 @@ public class TestHCM {
TableName tableName = TableName.valueOf("testClusterConnection");
TEST_UTIL.createTable(tableName, FAM_NAM).close();
- HTable t = (HTable)con1.getTable(tableName, otherPool);
- // make sure passing a pool to the getTable does not trigger creation of an internal pool
- assertNull("Internal Thread pool should be null",
- ((ConnectionImplementation) con1).getCurrentBatchPool());
- // table should use the pool passed
- assertTrue(otherPool == t.getPool());
- t.close();
-
- t = (HTable)con2.getTable(tableName);
- // table should use the connectin's internal pool
- assertTrue(otherPool == t.getPool());
- t.close();
+ Table table = con1.getTable(tableName, otherPool);
- t = (HTable)con2.getTable(tableName);
- // try other API too
- assertTrue(otherPool == t.getPool());
- t.close();
-
- t = (HTable)con2.getTable(tableName);
- // try other API too
- assertTrue(otherPool == t.getPool());
- t.close();
+ ExecutorService pool = null;
- t = (HTable)con1.getTable(tableName);
- ExecutorService pool = ((ConnectionImplementation)con1).getCurrentBatchPool();
- // make sure an internal pool was created
- assertNotNull("An internal Thread pool should have been created", pool);
- // and that the table is using it
- assertTrue(t.getPool() == pool);
- t.close();
-
- t = (HTable)con1.getTable(tableName);
- // still using the *same* internal pool
- assertTrue(t.getPool() == pool);
- t.close();
+ if(table instanceof HTable) {
+ HTable t = (HTable) table;
+ // make sure passing a pool to the getTable does not trigger creation of an internal pool
+ assertNull("Internal Thread pool should be null",
+ ((ConnectionImplementation) con1).getCurrentBatchPool());
+ // table should use the pool passed
+ assertTrue(otherPool == t.getPool());
+ t.close();
+
+ t = (HTable) con2.getTable(tableName);
+ // table should use the connectin's internal pool
+ assertTrue(otherPool == t.getPool());
+ t.close();
+
+ t = (HTable) con2.getTable(tableName);
+ // try other API too
+ assertTrue(otherPool == t.getPool());
+ t.close();
+
+ t = (HTable) con2.getTable(tableName);
+ // try other API too
+ assertTrue(otherPool == t.getPool());
+ t.close();
+
+ t = (HTable) con1.getTable(tableName);
+ pool = ((ConnectionImplementation) con1).getCurrentBatchPool();
+ // make sure an internal pool was created
+ assertNotNull("An internal Thread pool should have been created", pool);
+ // and that the table is using it
+ assertTrue(t.getPool() == pool);
+ t.close();
+
+ t = (HTable) con1.getTable(tableName);
+ // still using the *same* internal pool
+ assertTrue(t.getPool() == pool);
+ t.close();
+ } else {
+ table.close();
+ }
con1.close();
+
// if the pool was created on demand it should be closed upon connection close
- assertTrue(pool.isShutdown());
+ if(pool != null) {
+ assertTrue(pool.isShutdown());
+ }
con2.close();
// if the pool is passed, it is not closed
@@ -316,30 +327,27 @@ public class TestHCM {
public void testOperationTimeout() throws Exception {
HTableDescriptor hdt = TEST_UTIL.createTableDescriptor("HCM-testOperationTimeout");
hdt.addCoprocessor(SleepAndFailFirstTime.class.getName());
- Table t = TEST_UTIL.createTable(hdt, new byte[][]{FAM_NAM});
- if (t instanceof HTable) {
- HTable table = (HTable) t;
- table.setRpcTimeout(Integer.MAX_VALUE);
- // Check that it works if the timeout is big enough
- table.setOperationTimeout(120 * 1000);
+ Table table = TEST_UTIL.createTable(hdt, new byte[][]{FAM_NAM});
+ table.setRpcTimeout(Integer.MAX_VALUE);
+ // Check that it works if the timeout is big enough
+ table.setOperationTimeout(120 * 1000);
+ table.get(new Get(FAM_NAM));
+
+ // Resetting and retrying. Will fail this time, not enough time for the second try
+ SleepAndFailFirstTime.ct.set(0);
+ try {
+ table.setOperationTimeout(30 * 1000);
table.get(new Get(FAM_NAM));
-
- // Resetting and retrying. Will fail this time, not enough time for the second try
- SleepAndFailFirstTime.ct.set(0);
- try {
- table.setOperationTimeout(30 * 1000);
- table.get(new Get(FAM_NAM));
- Assert.fail("We expect an exception here");
- } catch (SocketTimeoutException e) {
- // The client has a CallTimeout class, but it's not shared.We're not very clean today,
- // in the general case you can expect the call to stop, but the exception may vary.
- // In this test however, we're sure that it will be a socket timeout.
- LOG.info("We received an exception, as expected ", e);
- } catch (IOException e) {
- Assert.fail("Wrong exception:" + e.getMessage());
- } finally {
- table.close();
- }
+ Assert.fail("We expect an exception here");
+ } catch (SocketTimeoutException e) {
+ // The client has a CallTimeout class, but it's not shared.We're not very clean today,
+ // in the general case you can expect the call to stop, but the exception may vary.
+ // In this test however, we're sure that it will be a socket timeout.
+ LOG.info("We received an exception, as expected ", e);
+ } catch (IOException e) {
+ Assert.fail("Wrong exception:" + e.getMessage());
+ } finally {
+ table.close();
}
}
@@ -350,11 +358,9 @@ public class TestHCM {
Configuration c = new Configuration(TEST_UTIL.getConfiguration());
try (Table t = TEST_UTIL.createTable(hdt, new byte[][] { FAM_NAM }, c)) {
- assert t instanceof HTable;
- HTable table = (HTable) t;
- table.setRpcTimeout(SleepCoprocessor.SLEEP_TIME / 2);
- table.setOperationTimeout(SleepCoprocessor.SLEEP_TIME * 100);
- table.get(new Get(FAM_NAM));
+ t.setRpcTimeout(SleepCoprocessor.SLEEP_TIME / 2);
+ t.setOperationTimeout(SleepCoprocessor.SLEEP_TIME * 100);
+ t.get(new Get(FAM_NAM));
}
}
@@ -373,29 +379,26 @@ public class TestHCM {
c.setInt(HConstants.HBASE_RPC_TIMEOUT_KEY, 4000);
Connection connection = ConnectionFactory.createConnection(c);
- Table t = connection.getTable(TableName.valueOf("HCM-testRpcRetryingCallerSleep"));
- if (t instanceof HTable) {
- HTable table = (HTable) t;
- table.setOperationTimeout(8000);
- // Check that it works. Because 2s + 3s * RETRY_BACKOFF[0] + 2s < 8s
- table.get(new Get(FAM_NAM));
+ Table table = connection.getTable(TableName.valueOf("HCM-testRpcRetryingCallerSleep"));
+ table.setOperationTimeout(8000);
+ // Check that it works. Because 2s + 3s * RETRY_BACKOFF[0] + 2s < 8s
+ table.get(new Get(FAM_NAM));
- // Resetting and retrying.
- SleepAndFailFirstTime.ct.set(0);
- try {
- table.setOperationTimeout(6000);
- // Will fail this time. After sleep, there are not enough time for second retry
- // Beacuse 2s + 3s + 2s > 6s
- table.get(new Get(FAM_NAM));
- Assert.fail("We expect an exception here");
- } catch (SocketTimeoutException e) {
- LOG.info("We received an exception, as expected ", e);
- } catch (IOException e) {
- Assert.fail("Wrong exception:" + e.getMessage());
- } finally {
- table.close();
- connection.close();
- }
+ // Resetting and retrying.
+ SleepAndFailFirstTime.ct.set(0);
+ try {
+ table.setOperationTimeout(6000);
+ // Will fail this time. After sleep, there are not enough time for second retry
+ // Beacuse 2s + 3s + 2s > 6s
+ table.get(new Get(FAM_NAM));
+ Assert.fail("We expect an exception here");
+ } catch (SocketTimeoutException e) {
+ LOG.info("We received an exception, as expected ", e);
+ } catch (IOException e) {
+ Assert.fail("Wrong exception:" + e.getMessage());
+ } finally {
+ table.close();
+ connection.close();
}
}
@@ -404,7 +407,7 @@ public class TestHCM {
long pauseTime;
long baseTime = 100;
TableName tableName = TableName.valueOf("HCM-testCallableSleep");
- HTable table = TEST_UTIL.createTable(tableName, FAM_NAM);
+ Table table = TEST_UTIL.createTable(tableName, FAM_NAM);
RegionServerCallable regionServerCallable = new RegionServerCallable(
TEST_UTIL.getConnection(), tableName, ROW) {
public Object call(int timeout) throws IOException {
@@ -882,15 +885,21 @@ public class TestHCM {
public void testConnectionManagement() throws Exception{
Table table0 = TEST_UTIL.createTable(TABLE_NAME1, FAM_NAM);
Connection conn = ConnectionFactory.createConnection(TEST_UTIL.getConfiguration());
- HTable table = (HTable) conn.getTable(TABLE_NAME1);
+ Table table = conn.getTable(TABLE_NAME1);
table.close();
assertFalse(conn.isClosed());
- assertFalse(table.getPool().isShutdown());
- table = (HTable) conn.getTable(TABLE_NAME1);
+ if(table instanceof HTable) {
+ assertFalse(((HTable) table).getPool().isShutdown());
+ }
+ table = conn.getTable(TABLE_NAME1);
table.close();
- assertFalse(table.getPool().isShutdown());
+ if(table instanceof HTable) {
+ assertFalse(((HTable) table).getPool().isShutdown());
+ }
conn.close();
- assertTrue(table.getPool().isShutdown());
+ if(table instanceof HTable) {
+ assertTrue(((HTable) table).getPool().isShutdown());
+ }
table0.close();
}
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestHTableMultiplexerFlushCache.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestHTableMultiplexerFlushCache.java
index 9be6b6c..8c54880 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestHTableMultiplexerFlushCache.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestHTableMultiplexerFlushCache.java
@@ -128,7 +128,7 @@ public class TestHTableMultiplexerFlushCache {
// Region cache (and not just tearing down the entire connection).
TableName TABLE = TableName.valueOf("testOnRegionMove");
final int NUM_REGIONS = 10;
- HTable htable = TEST_UTIL.createTable(TABLE, new byte[][] { FAMILY }, 3,
+ Table htable = TEST_UTIL.createTable(TABLE, new byte[][] { FAMILY }, 3,
Bytes.toBytes("aaaaa"), Bytes.toBytes("zzzzz"), NUM_REGIONS);
HTableMultiplexer multiplexer = new HTableMultiplexer(TEST_UTIL.getConfiguration(),
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestLeaseRenewal.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestLeaseRenewal.java
index 7170299..d8bc591 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestLeaseRenewal.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestLeaseRenewal.java
@@ -95,7 +95,7 @@ public class TestLeaseRenewal {
@Test
public void testLeaseRenewal() throws Exception {
- HTable table = TEST_UTIL.createTable(
+ Table table = TEST_UTIL.createTable(
TableName.valueOf("testLeaseRenewal"), FAMILY);
Put p = new Put(ROW_BYTES);
p.addColumn(FAMILY, COL_QUAL, VAL_BYTES);
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 5446570..7bd4f93 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
@@ -50,7 +50,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.Increment;
import org.apache.hadoop.hbase.client.Put;
import org.apache.hadoop.hbase.client.RegionLocator;
@@ -376,7 +375,7 @@ public class TestRegionObserverInterface {
@Test (timeout=300000)
public void testHBASE14489() throws IOException {
TableName tableName = TableName.valueOf("testHBASE14489");
- HTable table = util.createTable(tableName, new byte[][] { A });
+ Table table = util.createTable(tableName, new byte[][] { A });
Put put = new Put(ROW);
put.addColumn(A, A, A);
table.put(put);
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 59173ad..12761d3 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
@@ -66,7 +66,6 @@ import org.apache.hadoop.hbase.TagUtil;
import org.apache.hadoop.hbase.client.Admin;
import org.apache.hadoop.hbase.client.Connection;
import org.apache.hadoop.hbase.client.ConnectionFactory;
-import org.apache.hadoop.hbase.client.HTable;
import org.apache.hadoop.hbase.client.Put;
import org.apache.hadoop.hbase.client.RegionLocator;
import org.apache.hadoop.hbase.client.Result;
@@ -611,7 +610,7 @@ public class TestHFileOutputFormat2 {
Configuration conf = new Configuration(this.util.getConfiguration());
Map familyToCompression =
getMockColumnFamiliesForCompression(numCfs);
- Table table = Mockito.mock(HTable.class);
+ Table table = Mockito.mock(Table.class);
setupMockColumnFamiliesForCompression(table, familyToCompression);
HFileOutputFormat2.configureCompression(conf, table.getTableDescriptor());
@@ -682,7 +681,7 @@ public class TestHFileOutputFormat2 {
Configuration conf = new Configuration(this.util.getConfiguration());
Map familyToBloomType =
getMockColumnFamiliesForBloomType(numCfs);
- Table table = Mockito.mock(HTable.class);
+ Table table = Mockito.mock(Table.class);
setupMockColumnFamiliesForBloomType(table,
familyToBloomType);
HFileOutputFormat2.configureBloomType(table.getTableDescriptor(), conf);
@@ -753,7 +752,7 @@ public class TestHFileOutputFormat2 {
Configuration conf = new Configuration(this.util.getConfiguration());
Map familyToBlockSize =
getMockColumnFamiliesForBlockSize(numCfs);
- Table table = Mockito.mock(HTable.class);
+ Table table = Mockito.mock(Table.class);
setupMockColumnFamiliesForBlockSize(table,
familyToBlockSize);
HFileOutputFormat2.configureBlockSize(table.getTableDescriptor(), conf);
@@ -828,7 +827,7 @@ public class TestHFileOutputFormat2 {
Configuration conf = new Configuration(this.util.getConfiguration());
Map familyToDataBlockEncoding =
getMockColumnFamiliesForDataBlockEncoding(numCfs);
- Table table = Mockito.mock(HTable.class);
+ Table table = Mockito.mock(Table.class);
setupMockColumnFamiliesForDataBlockEncoding(table,
familyToDataBlockEncoding);
HTableDescriptor tableDescriptor = table.getTableDescriptor();
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/normalizer/TestSimpleRegionNormalizerOnCluster.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/normalizer/TestSimpleRegionNormalizerOnCluster.java
index 6ec2e95..f8bc6ab 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/normalizer/TestSimpleRegionNormalizerOnCluster.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/normalizer/TestSimpleRegionNormalizerOnCluster.java
@@ -28,8 +28,8 @@ import org.apache.hadoop.hbase.MiniHBaseCluster;
import org.apache.hadoop.hbase.NamespaceDescriptor;
import org.apache.hadoop.hbase.TableName;
import org.apache.hadoop.hbase.client.Admin;
-import org.apache.hadoop.hbase.client.HTable;
import org.apache.hadoop.hbase.client.Put;
+import org.apache.hadoop.hbase.client.Table;
import org.apache.hadoop.hbase.master.HMaster;
import org.apache.hadoop.hbase.master.TableNamespaceManager;
import org.apache.hadoop.hbase.master.normalizer.NormalizationPlan.PlanType;
@@ -104,7 +104,7 @@ public class TestSimpleRegionNormalizerOnCluster {
MiniHBaseCluster cluster = TEST_UTIL.getHBaseCluster();
HMaster m = cluster.getMaster();
- try (HTable ht = TEST_UTIL.createMultiRegionTable(TABLENAME, FAMILYNAME, 5)) {
+ try (Table ht = TEST_UTIL.createMultiRegionTable(TABLENAME, FAMILYNAME, 5)) {
// Need to get sorted list of regions here
List generatedRegions = TEST_UTIL.getHBaseCluster().getRegions(TABLENAME);
Collections.sort(generatedRegions, new Comparator() {
@@ -182,7 +182,7 @@ public class TestSimpleRegionNormalizerOnCluster {
HMaster m = cluster.getMaster();
// create 5 regions with sizes to trigger merge of small regions
- try (HTable ht = TEST_UTIL.createMultiRegionTable(TABLENAME, FAMILYNAME, 5)) {
+ try (Table ht = TEST_UTIL.createMultiRegionTable(TABLENAME, FAMILYNAME, 5)) {
// Need to get sorted list of regions here
List generatedRegions = TEST_UTIL.getHBaseCluster().getRegions(TABLENAME);
Collections.sort(generatedRegions, new Comparator() {
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestCorruptedRegionStoreFile.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestCorruptedRegionStoreFile.java
index 54dbe9b..68b0ba3 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestCorruptedRegionStoreFile.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestCorruptedRegionStoreFile.java
@@ -28,7 +28,6 @@ import org.apache.hadoop.fs.FileSystem;
import org.apache.hadoop.fs.Path;
import org.apache.hadoop.hbase.HBaseTestingUtility;
import org.apache.hadoop.hbase.TableName;
-import org.apache.hadoop.hbase.client.HTable;
import org.apache.hadoop.hbase.client.Scan;
import org.apache.hadoop.hbase.client.Result;
import org.apache.hadoop.hbase.client.ResultScanner;
@@ -92,13 +91,11 @@ public class TestCorruptedRegionStoreFile {
table.put(put);
if ((rowCount++ % ROW_PER_FILE) == 0) {
- // flush it
- ((HTable)table).flushCommits();
- UTIL.getHBaseAdmin().flush(tableName);
+ UTIL.getAdmin().flush(tableName);
}
}
} finally {
- UTIL.getHBaseAdmin().flush(tableName);
+ UTIL.getAdmin().flush(tableName);
table.close();
}
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 2087097..18796bd 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
@@ -558,9 +558,8 @@ public class TestRegionServerMetrics {
htd.addFamily(hcd);
Connection connection = ConnectionFactory.createConnection(conf);
Admin admin = connection.getAdmin();
- HTable t = TEST_UTIL.createTable(htd, new byte[0][0], conf);
+ Table t = TEST_UTIL.createTable(htd, new byte[0][0], conf);
Region region = rs.getOnlineRegions(tableName).get(0);
- t.setAutoFlush(true, true);
for (int insertCount = 0; insertCount < numHfiles; insertCount++) {
Put p = new Put(Bytes.toBytes(insertCount));
p.addColumn(cf, qualifier, val);
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/throttle/TestFlushWithThroughputController.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/throttle/TestFlushWithThroughputController.java
index 8908c71..ae6b036 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/throttle/TestFlushWithThroughputController.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/throttle/TestFlushWithThroughputController.java
@@ -28,7 +28,6 @@ import org.apache.hadoop.hbase.TableName;
import org.apache.hadoop.hbase.client.Connection;
import org.apache.hadoop.hbase.client.ConnectionFactory;
import org.apache.hadoop.hbase.client.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.regionserver.DefaultStoreEngine;
@@ -76,7 +75,7 @@ public class TestFlushWithThroughputController {
admin.disableTable(tableName);
admin.deleteTable(tableName);
}
- HTable table = TEST_UTIL.createTable(tableName, family);
+ Table table = TEST_UTIL.createTable(tableName, family);
Random rand = new Random();
for (int i = 0; i < 10; i++) {
for (int j = 0; j < 10; j++) {
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 da01fb9..bf46b03 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
@@ -39,7 +39,6 @@ import org.apache.hadoop.hbase.HConstants;
import org.apache.hadoop.hbase.HRegionInfo;
import org.apache.hadoop.hbase.HTableDescriptor;
import org.apache.hadoop.hbase.TableName;
-import org.apache.hadoop.hbase.client.HTable;
import org.apache.hadoop.hbase.client.Put;
import org.apache.hadoop.hbase.client.Result;
import org.apache.hadoop.hbase.client.ResultScanner;
@@ -53,7 +52,6 @@ import org.apache.hadoop.hbase.util.Bytes;
import org.apache.hadoop.hbase.util.FSUtils;
import org.apache.hadoop.hbase.util.JVMClusterUtil;
import org.apache.hadoop.hbase.wal.AbstractFSWALProvider;
-import org.apache.hadoop.hbase.wal.FSHLogProvider;
import org.apache.hadoop.hbase.wal.WAL;
import org.apache.hadoop.hbase.wal.WALFactory;
import org.apache.hadoop.hdfs.protocol.DatanodeInfo;
@@ -135,7 +133,6 @@ public class TestLogRolling extends AbstractTestLogRolling {
admin.createTable(desc);
Table table = TEST_UTIL.getConnection().getTable(desc.getTableName());
- assertTrue(((HTable) table).isAutoFlush());
server = TEST_UTIL.getRSForFirstRegionInTable(desc.getTableName());
HRegionInfo region = server.getOnlineRegions(desc.getTableName()).get(0).getRegionInfo();
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/security/token/TestTokenAuthentication.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/security/token/TestTokenAuthentication.java
index faac8eb..9382bd4 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/security/token/TestTokenAuthentication.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/security/token/TestTokenAuthentication.java
@@ -47,7 +47,7 @@ import org.apache.hadoop.hbase.TableName;
import org.apache.hadoop.hbase.client.ClusterConnection;
import org.apache.hadoop.hbase.client.Connection;
import org.apache.hadoop.hbase.client.ConnectionFactory;
-import org.apache.hadoop.hbase.client.HTableInterface;
+import org.apache.hadoop.hbase.client.Table;
import org.apache.hadoop.hbase.coprocessor.RegionCoprocessorEnvironment;
import org.apache.hadoop.hbase.ipc.BlockingRpcCallback;
import org.apache.hadoop.hbase.ipc.FifoRpcScheduler;
@@ -235,11 +235,11 @@ public class TestTokenAuthentication {
public Configuration getConfiguration() { return conf; }
@Override
- public HTableInterface getTable(TableName tableName) throws IOException
+ public Table getTable(TableName tableName) throws IOException
{ return null; }
@Override
- public HTableInterface getTable(TableName tableName, ExecutorService service)
+ public Table getTable(TableName tableName, ExecutorService service)
throws IOException {
return null;
}
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/tool/TestCanaryTool.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/tool/TestCanaryTool.java
index efca102..755e5ba 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/tool/TestCanaryTool.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/tool/TestCanaryTool.java
@@ -19,20 +19,14 @@
package org.apache.hadoop.hbase.tool;
-import org.apache.hadoop.conf.Configuration;
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.ServerName;
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.master.HMaster;
+import org.apache.hadoop.hbase.client.Table;
import org.apache.hadoop.hbase.testclassification.MediumTests;
import org.apache.hadoop.hbase.util.Bytes;
-import org.apache.hadoop.hbase.zookeeper.ZKUtil;
-import org.apache.hadoop.hbase.zookeeper.ZooKeeperWatcher;
import org.apache.hadoop.util.ToolRunner;
import org.apache.log4j.Appender;
import org.apache.log4j.LogManager;
@@ -87,7 +81,7 @@ public class TestCanaryTool {
@Test
public void testBasicCanaryWorks() throws Exception {
TableName tableName = TableName.valueOf("testTable");
- HTable table = testingUtility.createTable(tableName, new byte[][] { FAMILY });
+ Table table = testingUtility.createTable(tableName, new byte[][] { FAMILY });
// insert some test rows
for (int i=0; i<1000; i++) {
byte[] iBytes = Bytes.toBytes(i);
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 23b999e..77443e1 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
@@ -32,7 +32,6 @@ import org.apache.hadoop.hbase.client.ClusterConnection;
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;
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 84cc47d..cdf814c 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
@@ -26,7 +26,6 @@ import org.apache.commons.logging.LogFactory;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.hbase.TableName;
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.security.User;
@@ -69,7 +68,7 @@ public class MultiThreadedReaderWithACL extends MultiThreadedReader {
}
@Override
- protected HTableInterface createTable() throws IOException {
+ protected Table createTable() throws IOException {
return null;
}
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 0f3baf9..6c816cf 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
@@ -40,7 +40,6 @@ import org.apache.hadoop.hbase.TableName;
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.HTableInterface;
import org.apache.hadoop.hbase.client.Increment;
import org.apache.hadoop.hbase.client.Mutation;
import org.apache.hadoop.hbase.client.Put;
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 756f612..bf27dde 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
@@ -31,7 +31,6 @@ import org.apache.hadoop.hbase.TableName;
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.HTableInterface;
import org.apache.hadoop.hbase.client.Increment;
import org.apache.hadoop.hbase.client.Mutation;
import org.apache.hadoop.hbase.client.Put;
@@ -84,7 +83,7 @@ public class MultiThreadedUpdaterWithACL extends MultiThreadedUpdater {
}
@Override
- protected HTableInterface createTable() throws IOException {
+ protected Table createTable() throws IOException {
return null;
}
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 32a06bb..d53ab25 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
@@ -33,7 +33,6 @@ import org.apache.commons.logging.LogFactory;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.hbase.HConstants;
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;
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 d3cba2b..4806288 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
@@ -26,7 +26,6 @@ import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
import org.apache.hadoop.conf.Configuration;
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;
@@ -71,7 +70,7 @@ public class MultiThreadedWriterWithACL extends MultiThreadedWriter {
}
@Override
- protected HTableInterface createTable() throws IOException {
+ protected Table createTable() throws IOException {
return null;
}
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 9f60d61..9dea9a5 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
@@ -74,7 +74,7 @@ import org.apache.thrift.TException;
/**
* This class is a glue object that connects Thrift RPC calls to the HBase client API primarily
- * defined in the HTableInterface.
+ * defined in the Table interface.
*/
@InterfaceAudience.Private
@SuppressWarnings("deprecation")
--
2.5.0
| | |
|