From 490fe430f45c9ea933b020fe6a090d6d54cd164e Mon Sep 17 00:00:00 2001 From: Jurriaan Mous Date: Sun, 22 May 2016 14:22:23 +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 | 31 +-- .../apache/hadoop/hbase/client/HConnection.java | 111 +------- .../org/apache/hadoop/hbase/client/HTable.java | 57 +--- .../hadoop/hbase/client/HTableInterface.java | 138 ---------- .../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 | 19 -- .../mob/compactions/PartitionedMobCompactor.java | 49 ++-- .../main/resources/hbase-webapps/master/table.jsp | 14 +- .../apache/hadoop/hbase/HBaseTestingUtility.java | 65 +++-- .../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 +- .../hbase/client/TestShortCircuitConnection.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 | 3 +- .../hbase/util/MultiThreadedReaderWithACL.java | 3 +- .../hadoop/hbase/util/MultiThreadedUpdater.java | 3 +- .../hbase/util/MultiThreadedUpdaterWithACL.java | 3 +- .../hadoop/hbase/util/MultiThreadedWriter.java | 3 +- .../hbase/util/MultiThreadedWriterWithACL.java | 3 +- .../hbase/thrift2/ThriftHBaseServiceHandler.java | 2 +- 40 files changed, 410 insertions(+), 788 deletions(-) delete mode 100644 hbase-client/src/main/java/org/apache/hadoop/hbase/client/HTableInterface.java 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 429e47d..44d9f24 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 @@ -147,10 +147,10 @@ class ConnectionImplementation implements ClusterConnection, Closeable { // be waiting for the master lock => deadlock. private final Object masterAndZKLock = new Object(); - // thread executor shared by all HTableInterface instances created + // thread executor shared by all HTable instances created // by this connection private volatile ExecutorService batchPool = null; - // meta thread executor shared by all HTableInterface instances created + // meta thread executor shared by all HTable instances created // by this connection private volatile ExecutorService metaLookupPool = null; private volatile boolean cleanupPool = false; @@ -279,34 +279,13 @@ class ConnectionImplementation implements ClusterConnection, Closeable { return ng; } - - @Override - public HTableInterface getTable(String tableName) throws IOException { - return getTable(TableName.valueOf(tableName)); - } - - @Override - public HTableInterface getTable(byte[] tableName) throws IOException { - return getTable(TableName.valueOf(tableName)); - } - @Override - public HTableInterface getTable(TableName tableName) throws IOException { + public Table getTable(TableName tableName) throws IOException { return getTable(tableName, getBatchPool()); } @Override - public HTableInterface getTable(String tableName, ExecutorService pool) throws IOException { - return getTable(TableName.valueOf(tableName), pool); - } - - @Override - public HTableInterface getTable(byte[] tableName, ExecutorService pool) throws IOException { - return getTable(TableName.valueOf(tableName), pool); - } - - @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); } @@ -1923,7 +1902,7 @@ class ConnectionImplementation implements ClusterConnection, Closeable { } /** - * @deprecated since 0.96 Use {@link org.apache.hadoop.hbase.client.HTableInterface#batch} instead + * @deprecated since 0.96 Use {@link org.apache.hadoop.hbase.client.Table#batch} instead */ @Override @Deprecated diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/HConnection.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/HConnection.java index cc5e9fa..8138a61 100644 --- a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/HConnection.java +++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/HConnection.java @@ -58,113 +58,8 @@ public interface HConnection extends Connection { * Key for configuration in Configuration whose value is the class we implement making a * new HConnection instance. */ - public static final String HBASE_CLIENT_CONNECTION_IMPL = "hbase.client.connection.impl"; + String HBASE_CLIENT_CONNECTION_IMPL = "hbase.client.connection.impl"; - /** - * @return Configuration instance being used by this HConnection instance. - */ - @Override - Configuration getConfiguration(); - - /** - * Retrieve an HTableInterface implementation for access to a table. - * The returned HTableInterface is not thread safe, a new instance should - * be created for each using thread. - * This is a lightweight operation, pooling or caching of the returned HTableInterface - * is neither required nor desired. - * (created with {@link ConnectionFactory#createConnection(Configuration)}). - * @param tableName - * @return an HTable to use for interactions with this table - */ - public HTableInterface getTable(String tableName) throws IOException; - - /** - * Retrieve an HTableInterface implementation for access to a table. - * The returned HTableInterface is not thread safe, a new instance should - * be created for each using thread. - * This is a lightweight operation, pooling or caching of the returned HTableInterface - * is neither required nor desired. - * (created with {@link ConnectionFactory#createConnection(Configuration)}). - * @param tableName - * @return an HTable to use for interactions with this table - */ - public HTableInterface getTable(byte[] tableName) throws IOException; - - /** - * Retrieve an HTableInterface implementation for access to a table. - * The returned HTableInterface is not thread safe, a new instance should - * be created for each using thread. - * This is a lightweight operation, pooling or caching of the returned HTableInterface - * is neither required nor desired. - * (created with {@link ConnectionFactory#createConnection(Configuration)}). - * @param tableName - * @return an HTable to use for interactions with this table - */ - @Override - public HTableInterface getTable(TableName tableName) throws IOException; - - /** - * Retrieve an HTableInterface implementation for access to a table. - * The returned HTableInterface is not thread safe, a new instance should - * be created for each using thread. - * This is a lightweight operation, pooling or caching of the returned HTableInterface - * is neither required nor desired. - * (created with {@link ConnectionFactory#createConnection(Configuration)}). - * @param tableName - * @param pool The thread pool to use for batch operations, null to use a default pool. - * @return an HTable to use for interactions with this table - */ - public HTableInterface getTable(String tableName, ExecutorService pool) throws IOException; - - /** - * Retrieve an HTableInterface implementation for access to a table. - * The returned HTableInterface is not thread safe, a new instance should - * be created for each using thread. - * This is a lightweight operation, pooling or caching of the returned HTableInterface - * is neither required nor desired. - * (created with {@link ConnectionFactory#createConnection(Configuration)}). - * @param tableName - * @param pool The thread pool to use for batch operations, null to use a default pool. - * @return an HTable to use for interactions with this table - */ - public HTableInterface getTable(byte[] tableName, ExecutorService pool) throws IOException; - - /** - * Retrieve an HTableInterface implementation for access to a table. - * The returned HTableInterface is not thread safe, a new instance should - * be created for each using thread. - * This is a lightweight operation, pooling or caching of the returned HTableInterface - * is neither required nor desired. - * (created with {@link ConnectionFactory#createConnection(Configuration)}). - * @param tableName table to get interface for - * @param pool The thread pool to use for batch operations, null to use a default pool. - * @return an HTable to use for interactions with this table - */ - @Override - public HTableInterface getTable(TableName tableName, ExecutorService pool) throws IOException; - - /** - * Retrieve a RegionLocator implementation to inspect region information on a table. The returned - * RegionLocator is not thread-safe, so a new instance should be created for each using thread. - * - * This is a lightweight operation. Pooling or caching of the returned RegionLocator is neither - * required nor desired. - * @param tableName Name of the table who's region is to be examined - * @return A RegionLocator instance - */ - @Override - public RegionLocator getRegionLocator(TableName tableName) throws IOException; - - /** - * Retrieve an Admin implementation to administer an HBase cluster. - * The returned Admin is not guaranteed to be thread-safe. A new instance should be created for - * each using thread. This is a lightweight operation. Pooling or caching of the returned - * Admin is not recommended. - * - * @return an Admin instance for cluster administration - */ - @Override - Admin getAdmin() throws IOException; /** @return - true if the master server is running * @deprecated internal method, do not use thru HConnection */ @@ -500,7 +395,7 @@ public interface HConnection extends Connection { * processed successfully. * @throws IOException if there are problems talking to META. Per-item * exceptions are stored in the results array. - * @deprecated since 0.96 - Use {@link HTableInterface#batch} instead + * @deprecated since 0.96 - Use {@link Table#batch} instead */ @Deprecated void processBatch(List actions, final TableName tableName, @@ -516,7 +411,7 @@ public interface HConnection extends Connection { /** * Parameterized batch processing, allowing varying return types for different * {@link Row} implementations. - * @deprecated since 0.96 - Use {@link HTableInterface#batchCallback} instead + * @deprecated since 0.96 - Use {@link Table#batchCallback} instead */ @Deprecated public void processBatchCallback(List list, 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 bf9ec22..33fb7c2 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 com.google.protobuf.ServiceException; */ @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; @@ -254,12 +245,9 @@ public class HTable implements HTableInterface { * INTERNAL Used by unit tests and tools to do low-level * manipulations. * @return An HConnection instance. - * @deprecated This method will be changed from public to package protected. */ - // TODO(tsuna): Remove this. Unit tests shouldn't require public helpers. - @Deprecated @VisibleForTesting - public HConnection getConnection() { + protected HConnection getConnection() { return this.connection; } @@ -331,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 { @@ -382,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 { @@ -393,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) @@ -564,9 +552,7 @@ public class HTable implements HTableInterface { @Override public void put(final Put put) throws IOException { getBufferedMutator().mutate(put); - if (autoFlush) { - flushCommits(); - } + flushCommits(); } /** @@ -576,9 +562,7 @@ public class HTable implements HTableInterface { @Override public void put(final List puts) throws IOException { getBufferedMutator().mutate(puts); - if (autoFlush) { - flushCommits(); - } + flushCommits(); } /** @@ -976,8 +960,7 @@ public class HTable implements HTableInterface { * {@inheritDoc} * @throws IOException */ - @Override - public void flushCommits() throws IOException { + private void flushCommits() throws IOException { if (mutator == null) { // nothing to flush if there's no mutator; don't bother creating one. return; @@ -1062,30 +1045,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/HTableInterface.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/HTableInterface.java deleted file mode 100644 index 4cd81e7..0000000 --- a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/HTableInterface.java +++ /dev/null @@ -1,138 +0,0 @@ -/** - * - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ -package org.apache.hadoop.hbase.client; - -import java.io.IOException; - -import org.apache.hadoop.hbase.classification.InterfaceAudience; -import org.apache.hadoop.hbase.classification.InterfaceStability; - -/** - * Used to communicate with a single HBase table. - * Obtain an instance from an {@link HConnection}. - * - * @since 0.21.0 - * @deprecated use {@link org.apache.hadoop.hbase.client.Table} instead - */ -@Deprecated -@InterfaceAudience.Private -@InterfaceStability.Stable -public interface HTableInterface extends Table { - - /** - * Gets the name of this table. - * - * @return the table name. - * @deprecated Use {@link #getName()} instead - */ - @Deprecated - byte[] getTableName(); - - /** - * Turns 'auto-flush' on or off. - *

- * When enabled (default), {@link Put} operations don't get buffered/delayed - * and are immediately executed. Failed operations are not retried. This is - * slower but safer. - *

- * Turning off {@code #autoFlush} means that multiple {@link Put}s will be - * accepted before any RPC is actually sent to do the write operations. If the - * application dies before pending writes get flushed to HBase, data will be - * lost. - *

- * When you turn {@code #autoFlush} off, you should also consider the - * {@code #clearBufferOnFail} option. By default, asynchronous {@link Put} - * requests will be retried on failure until successful. However, this can - * pollute the writeBuffer and slow down batching performance. Additionally, - * you may want to issue a number of Put requests and call - * {@link #flushCommits()} as a barrier. In both use cases, consider setting - * clearBufferOnFail to true to erase the buffer after {@link #flushCommits()} - * has been called, regardless of success. - *

- * In other words, if you call {@code #setAutoFlush(false)}; HBase will retry N time for each - * flushCommit, including the last one when closing the table. This is NOT recommended, - * most of the time you want to call {@code #setAutoFlush(false, true)}. - * - * @param autoFlush - * Whether or not to enable 'auto-flush'. - * @param clearBufferOnFail - * Whether to keep Put failures in the writeBuffer. If autoFlush is true, then - * the value of this parameter is ignored and clearBufferOnFail is set to true. - * Setting clearBufferOnFail to false is deprecated since 0.96. - * @deprecated in 0.99 since setting clearBufferOnFail is deprecated. - * @see BufferedMutator#flush() - */ - @Deprecated - void setAutoFlush(boolean autoFlush, boolean clearBufferOnFail); - - /** - * Set the autoFlush behavior, without changing the value of {@code clearBufferOnFail}. - * @deprecated in 0.99 since setting clearBufferOnFail is deprecated. Move on to - * {@link BufferedMutator} - */ - @Deprecated - void setAutoFlushTo(boolean autoFlush); - - /** - * Tells whether or not 'auto-flush' is turned on. - * - * @return {@code true} if 'auto-flush' is enabled (default), meaning - * {@link Put} operations don't get buffered/delayed and are immediately - * executed. - * @deprecated as of 1.0.0. Replaced by {@link BufferedMutator} - */ - @Deprecated - boolean isAutoFlush(); - - /** - * Executes all the buffered {@link Put} operations. - *

- * This method gets called once automatically for every {@link Put} or batch - * of {@link Put}s (when put(List<Put>) is used) when - * {@link #isAutoFlush} is {@code true}. - * @throws IOException if a remote or network exception occurs. - * @deprecated as of 1.0.0. Replaced by {@link BufferedMutator#flush()} - */ - @Deprecated - void flushCommits() throws IOException; - - /** - * Returns the maximum size in bytes of the write buffer for this HTable. - *

- * The default value comes from the configuration parameter - * {@code hbase.client.write.buffer}. - * @return The size of the write buffer in bytes. - * @deprecated as of 1.0.0. Replaced by {@link BufferedMutator#getWriteBufferSize()} - */ - @Deprecated - long getWriteBufferSize(); - - /** - * Sets the size of the buffer in bytes. - *

- * If the new size is less than the current amount of data in the - * write buffer, the buffer gets flushed. - * @param writeBufferSize The new write buffer size, in bytes. - * @throws IOException if a remote or network exception occurs. - * @deprecated as of 1.0.0. Replaced by {@link BufferedMutator} and - * {@link BufferedMutatorParams#writeBufferSize(long)} - */ - @Deprecated - void setWriteBufferSize(long writeBufferSize) throws IOException; -} 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 0084878..6929c97 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 @@ -45,7 +45,6 @@ import org.apache.hadoop.hbase.classification.InterfaceStability; import org.apache.hadoop.hbase.client.Admin; import org.apache.hadoop.hbase.client.Connection; import org.apache.hadoop.hbase.client.ConnectionFactory; -import org.apache.hadoop.hbase.client.HTable; import org.apache.hadoop.hbase.client.RegionLocator; import org.apache.hadoop.hbase.client.RegionServerCallable; import org.apache.hadoop.hbase.client.RpcRetryingCallerFactory; @@ -312,24 +311,6 @@ public class LoadIncrementalHFiles extends Configured implements Tool { * @param table the table to load into * @throws TableNotFoundException if table does not yet exist */ - @SuppressWarnings("deprecation") - public void doBulkLoad(Path hfofDir, final HTable table) - throws TableNotFoundException, IOException { - try (Admin admin = table.getConnection().getAdmin(); - RegionLocator rl = table.getRegionLocator()) { - doBulkLoad(hfofDir, admin, table, rl); - } - } - - /** - * Perform a bulk load of the given directory into the given - * pre-existing table. This method is not threadsafe. - * - * @param hfofDir the directory that was provided as the output path - * of a job using HFileOutputFormat - * @param table the table to load into - * @throws TableNotFoundException if table does not yet exist - */ public void doBulkLoad(Path hfofDir, final Admin admin, Table table, RegionLocator regionLocator) throws TableNotFoundException, IOException { 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 cfe76ae..60d02bb 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 @@ -50,7 +50,6 @@ import org.apache.hadoop.hbase.TagType; import org.apache.hadoop.hbase.classification.InterfaceAudience; import org.apache.hadoop.hbase.client.Connection; import org.apache.hadoop.hbase.client.ConnectionFactory; -import org.apache.hadoop.hbase.client.HTable; import org.apache.hadoop.hbase.client.Scan; import org.apache.hadoop.hbase.client.Table; import org.apache.hadoop.hbase.io.HFileLink; @@ -257,7 +256,7 @@ public class PartitionedMobCompactor extends MobCompactor { return Collections.emptyList(); } List paths = new ArrayList(); - Connection c = ConnectionFactory.createConnection(conf); + final Connection c = ConnectionFactory.createConnection(conf); final Table table = c.getTable(tableName); try { Map>> results = @@ -268,7 +267,7 @@ public class PartitionedMobCompactor extends MobCompactor { @Override public List call() throws Exception { LOG.info("Compacting mob files for partition " + partition.getPartitionId()); - return compactMobFilePartition(request, partition, delFiles, table); + return compactMobFilePartition(request, partition, delFiles, c, table); } })); } @@ -291,7 +290,7 @@ public class PartitionedMobCompactor extends MobCompactor { try { table.close(); } catch (IOException e) { - LOG.error("Failed to close the HTable", e); + LOG.error("Failed to close the Table", e); } } return paths; @@ -302,12 +301,15 @@ public class PartitionedMobCompactor extends MobCompactor { * @param request The compaction request. * @param partition A compaction partition. * @param delFiles The del files. - * @param table The current table. - * @return The paths of new mob files after compactions. + * @param connection to use + * @param table The current table. @return The paths of new mob files after compactions. * @throws IOException */ private List compactMobFilePartition(PartitionedMobCompactionRequest request, - CompactionPartition partition, List delFiles, Table table) throws IOException { + CompactionPartition partition, + List delFiles, + Connection connection, + Table table) throws IOException { List newFiles = new ArrayList(); List files = partition.listFiles(); int offset = 0; @@ -336,7 +338,7 @@ public class PartitionedMobCompactor extends MobCompactor { } filesToCompact.addAll(delFiles); // compact the mob files in a batch. - compactMobFilesInBatch(request, partition, table, filesToCompact, batch, + compactMobFilesInBatch(request, partition, connection, table, filesToCompact, batch, bulkloadPathOfPartition, bulkloadColumnPath, newFiles); // move to the next batch. offset += batch; @@ -364,19 +366,22 @@ public class PartitionedMobCompactor extends MobCompactor { * Compacts a partition of selected small mob files and all the del files in a batch. * @param request The compaction request. * @param partition A compaction partition. + * @param connection To use for transport * @param table The current table. * @param filesToCompact The files to be compacted. * @param batch The number of mob files to be compacted in a batch. * @param bulkloadPathOfPartition The directory where the bulkload column of the current - * partition is saved. +* partition is saved. * @param bulkloadColumnPath The directory where the bulkload files of current partition - * are saved. - * @param newFiles The paths of new mob files after compactions. - * @throws IOException +* are saved. + * @param newFiles The paths of new mob files after compactions. @throws IOException */ private void compactMobFilesInBatch(PartitionedMobCompactionRequest request, - CompactionPartition partition, Table table, List filesToCompact, int batch, - Path bulkloadPathOfPartition, Path bulkloadColumnPath, List newFiles) + CompactionPartition partition, + Connection connection, Table table, + List filesToCompact, int batch, + Path bulkloadPathOfPartition, Path bulkloadColumnPath, + List newFiles) throws IOException { // open scanner to the selected mob files and del files. StoreScanner scanner = createScanner(filesToCompact, ScanType.COMPACT_DROP_DELETES); @@ -400,8 +405,8 @@ public class PartitionedMobCompactor extends MobCompactor { refFileWriter = MobUtils.createRefFileWriter(conf, fs, column, bulkloadColumnPath, fileInfo .getSecond().longValue(), compactionCacheConfig, cryptoContext); refFilePath = refFileWriter.getPath(); - List cells = new ArrayList(); - boolean hasMore = false; + List cells = new ArrayList<>(); + boolean hasMore; ScannerContext scannerContext = ScannerContext.newBuilder().setBatchLimit(compactionKVMax).build(); do { @@ -428,7 +433,7 @@ public class PartitionedMobCompactor extends MobCompactor { // commit mob file MobUtils.commitFile(conf, fs, filePath, mobFamilyDir, compactionCacheConfig); // bulkload the ref file - bulkloadRefFile(table, bulkloadPathOfPartition, filePath.getName()); + bulkloadRefFile(connection, table, bulkloadPathOfPartition, filePath.getName()); newFiles.add(new Path(mobFamilyDir, filePath.getName())); } else { // remove the new files @@ -561,17 +566,23 @@ public class PartitionedMobCompactor extends MobCompactor { /** * Bulkloads the current file. + * + * @param connection * @param table The current table. * @param bulkloadDirectory The path of bulkload directory. * @param fileName The current file name. * @throws IOException */ - private void bulkloadRefFile(Table table, Path bulkloadDirectory, String fileName) + private void bulkloadRefFile(Connection connection, Table table, Path bulkloadDirectory, + String fileName) throws IOException { // bulkload the ref file try { LoadIncrementalHFiles bulkload = new LoadIncrementalHFiles(conf); - bulkload.doBulkLoad(bulkloadDirectory, (HTable)table); + bulkload.doBulkLoad(bulkloadDirectory, + connection.getAdmin(), + table, + connection.getRegionLocator(table.getName())); } catch (Exception e) { // delete the committed mob file deletePath(new Path(mobFamilyDir, fileName)); 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..062d723 100644 --- a/hbase-server/src/main/resources/hbase-webapps/master/table.jsp +++ b/hbase-server/src/main/resources/hbase-webapps/master/table.jsp @@ -32,11 +32,7 @@ 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.HRegionInfo" import="org.apache.hadoop.hbase.HRegionLocation" import="org.apache.hadoop.hbase.ServerName" import="org.apache.hadoop.hbase.ServerLoad" @@ -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.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

"; 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 59b5bb3..aecc8af 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 @@ -71,7 +71,6 @@ import org.apache.hadoop.hbase.client.Durability; import org.apache.hadoop.hbase.client.Get; import org.apache.hadoop.hbase.client.HBaseAdmin; import org.apache.hadoop.hbase.client.HConnection; -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; @@ -1331,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}); } @@ -1344,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"); @@ -1361,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); } @@ -1373,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); } @@ -1385,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); @@ -1403,7 +1402,7 @@ public class HBaseTestingUtility extends HBaseCommonTestingUtility { // 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); } /** @@ -1414,7 +1413,7 @@ 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); } @@ -1428,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); @@ -1442,7 +1441,7 @@ public class HBaseTestingUtility extends HBaseCommonTestingUtility { // 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()); } /** @@ -1452,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); // 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()); } /** @@ -1470,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); } @@ -1483,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); } @@ -1496,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); } @@ -1510,7 +1509,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, byte[][] splitKeys) throws IOException { HTableDescriptor desc = new HTableDescriptor(tableName); for (byte[] family : families) { @@ -1521,7 +1520,7 @@ public class HBaseTestingUtility extends HBaseCommonTestingUtility { // 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); } /** @@ -1532,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); } @@ -1546,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) { @@ -1559,10 +1558,10 @@ public class HBaseTestingUtility extends HBaseCommonTestingUtility { // 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) { @@ -1578,7 +1577,7 @@ public class HBaseTestingUtility extends HBaseCommonTestingUtility { // 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); } /** @@ -1589,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); @@ -1604,7 +1603,7 @@ public class HBaseTestingUtility extends HBaseCommonTestingUtility { // 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); } /** @@ -1615,7 +1614,7 @@ 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); @@ -1624,7 +1623,7 @@ public class HBaseTestingUtility extends HBaseCommonTestingUtility { // 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); } /** @@ -1634,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); } @@ -1874,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) { @@ -1894,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(); if (!admin.isTableDisabled(tableName)) { admin.disableTable(tableName); } admin.truncateTable(tableName, preserveRegions); - return (HTable) getConnection().getTable(tableName); + return getConnection().getTable(tableName); } /** @@ -1913,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); } 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 24ef5b2..29fba6e 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..9d9dc95 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 @@ -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); @@ -136,7 +136,11 @@ public class TestClientPushback { final AtomicLong endTime = new AtomicLong(); long startTime = EnvironmentEdgeManager.currentTime(); - table.mutator.ap.submit(tableName, ops, true, new Batch.Callback() { + + BufferedMutatorImpl mutator = + (BufferedMutatorImpl) UTIL.getConnection().getBufferedMutator(tableName); + + 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 +176,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 b35a58f..c52dccb 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 8350b60..07ea58a 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/client/TestShortCircuitConnection.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestShortCircuitConnection.java index 618717b..c93794d 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestShortCircuitConnection.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestShortCircuitConnection.java @@ -66,7 +66,7 @@ public class TestShortCircuitConnection { UTIL.createTable(htd, null); HRegionServer regionServer = UTIL.getRSForFirstRegionInTable(tn); ClusterConnection connection = regionServer.getClusterConnection(); - HTableInterface tableIf = connection.getTable(tn); + Table tableIf = connection.getTable(tn); assertTrue(tableIf instanceof HTable); HTable table = (HTable) tableIf; assertTrue(table.getConnection() == connection); 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 ca06e97..0b7c20a 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 @@ -31,7 +31,6 @@ import org.apache.hadoop.hbase.TableName; 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; @@ -158,7 +157,7 @@ public class MultiThreadedReader extends MultiThreadedAction setName(getClass().getSimpleName() + "_" + readerId); } - protected HTableInterface createTable() throws IOException { + protected Table createTable() throws IOException { return connection.getTable(tableName); } 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 e28acc6..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; @@ -130,7 +129,7 @@ public class MultiThreadedUpdater extends MultiThreadedWriterBase { table = createTable(); } - protected HTableInterface createTable() throws IOException { + protected Table createTable() throws IOException { return connection.getTable(tableName); } 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 83e207a..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; @@ -87,7 +86,7 @@ public class MultiThreadedWriter extends MultiThreadedWriterBase { table = createTable(); } - protected HTableInterface createTable() throws IOException { + protected Table createTable() throws IOException { return connection.getTable(tableName); } 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
NameRegion ServerReadRequestsWriteRequestsStorefileSizeNum.StorefilesMemSizeLocalityStart KeyEnd KeyReplicaID