From 8fe52c83cb8c2ba3d74761e211846f249f5b5a52 Mon Sep 17 00:00:00 2001 From: Ashish Singhi Date: Wed, 17 Jun 2015 16:12:18 +0530 Subject: [PATCH] HBASE-13214 Remove deprecated and unused methods from HTable class --- .../hadoop/hbase/client/BufferedMutator.java | 4 - .../org/apache/hadoop/hbase/client/HTable.java | 397 +------------ .../hadoop/hbase/client/HTableInterface.java | 47 +- .../java/org/apache/hadoop/hbase/client/Table.java | 18 +- .../hadoop/hbase/rest/client/RemoteHTable.java | 5 - .../apache/hadoop/hbase/client/HTableWrapper.java | 12 - .../replication/regionserver/ReplicationSink.java | 2 +- .../main/resources/hbase-webapps/master/table.jsp | 32 +- .../org/apache/hadoop/hbase/client/TestAdmin1.java | 622 +++++++++++---------- .../org/apache/hadoop/hbase/client/TestAdmin2.java | 24 +- .../hadoop/hbase/client/TestFromClientSide.java | 358 +++++++----- .../hadoop/hbase/client/TestFromClientSide3.java | 243 ++++---- .../hbase/client/TestFromClientSideNoCodec.java | 5 +- .../org/apache/hadoop/hbase/client/TestHCM.java | 40 +- .../hadoop/hbase/client/TestHTableMultiplexer.java | 100 ++-- .../client/TestHTableMultiplexerFlushCache.java | 44 +- .../hadoop/hbase/client/TestMultiParallel.java | 37 +- .../hadoop/hbase/client/TestReplicasClient.java | 20 +- .../hbase/client/TestScannersFromClientSide.java | 64 ++- .../client/TestSnapshotCloneIndependence.java | 20 +- .../hbase/coprocessor/TestCoprocessorEndpoint.java | 42 +- .../hbase/coprocessor/TestHTableWrapper.java | 18 +- .../hbase/coprocessor/TestMasterObserver.java | 21 +- .../coprocessor/TestOpenTableInCoprocessor.java | 4 +- .../coprocessor/TestRegionObserverInterface.java | 23 +- .../io/encoding/TestLoadAndSwitchEncodeOnDisk.java | 29 +- .../hbase/mapreduce/TestHFileOutputFormat.java | 10 +- .../hadoop/hbase/master/TestRegionPlacement.java | 16 +- .../hbase/regionserver/TestHRegionOnCluster.java | 24 +- .../hbase/regionserver/TestRegionReplicas.java | 21 +- .../regionserver/TestRegionServerNoMaster.java | 5 +- .../TestRegionServerOnlineConfigChange.java | 16 +- .../regionserver/TestServerCustomProtocol.java | 30 +- ...stRegionReplicaReplicationEndpointNoMaster.java | 16 +- .../apache/hadoop/hbase/util/TestHBaseFsck.java | 182 +++--- .../hadoop/hbase/util/TestRegionSplitter.java | 52 +- .../util/hbck/OfflineMetaRebuildTestCore.java | 41 +- 37 files changed, 1207 insertions(+), 1437 deletions(-) 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 4424cec..355840c 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 @@ -51,10 +51,6 @@ import java.util.List; * also be effectively used in high volume online systems to batch puts, with the caveat that * 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 - * {@link HTableInterface#setAutoFlush(boolean)} set to {@code false}. - *

- * *

See also the {@code BufferedMutatorExample} in the hbase-examples module.

* @see ConnectionFactory * @see Connection 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 686aaa8..4efe1ef 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 @@ -24,7 +24,6 @@ import java.util.ArrayList; import java.util.Collections; import java.util.List; import java.util.Map; -import java.util.NavigableMap; import java.util.TreeMap; import java.util.concurrent.Callable; import java.util.concurrent.ExecutionException; @@ -38,14 +37,10 @@ import org.apache.commons.logging.Log; import org.apache.commons.logging.LogFactory; import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.hbase.Cell; -import org.apache.hadoop.hbase.HBaseConfiguration; import org.apache.hadoop.hbase.HConstants; -import org.apache.hadoop.hbase.HRegionInfo; import org.apache.hadoop.hbase.HRegionLocation; import org.apache.hadoop.hbase.HTableDescriptor; import org.apache.hadoop.hbase.KeyValueUtil; -import org.apache.hadoop.hbase.MetaTableAccessor; -import org.apache.hadoop.hbase.ServerName; import org.apache.hadoop.hbase.TableName; import org.apache.hadoop.hbase.TableNotFoundException; import org.apache.hadoop.hbase.classification.InterfaceAudience; @@ -241,123 +236,6 @@ public class HTable implements HTableInterface { } /** - * Tells whether or not a table is enabled or not. This method creates a - * new HBase configuration, so it might make your unit tests fail due to - * incorrect ZK client port. - * @param tableName Name of table to check. - * @return {@code true} if table is online. - * @throws IOException if a remote or network exception occurs - * @deprecated use {@link HBaseAdmin#isTableEnabled(byte[])} - */ - @Deprecated - public static boolean isTableEnabled(String tableName) throws IOException { - return isTableEnabled(TableName.valueOf(tableName)); - } - - /** - * Tells whether or not a table is enabled or not. This method creates a - * new HBase configuration, so it might make your unit tests fail due to - * incorrect ZK client port. - * @param tableName Name of table to check. - * @return {@code true} if table is online. - * @throws IOException if a remote or network exception occurs - * @deprecated use {@link HBaseAdmin#isTableEnabled(byte[])} - */ - @Deprecated - public static boolean isTableEnabled(byte[] tableName) throws IOException { - return isTableEnabled(TableName.valueOf(tableName)); - } - - /** - * Tells whether or not a table is enabled or not. This method creates a - * new HBase configuration, so it might make your unit tests fail due to - * incorrect ZK client port. - * @param tableName Name of table to check. - * @return {@code true} if table is online. - * @throws IOException if a remote or network exception occurs - * @deprecated use {@link HBaseAdmin#isTableEnabled(byte[])} - */ - @Deprecated - public static boolean isTableEnabled(TableName tableName) throws IOException { - return isTableEnabled(HBaseConfiguration.create(), tableName); - } - - /** - * Tells whether or not a table is enabled or not. - * @param conf The Configuration object to use. - * @param tableName Name of table to check. - * @return {@code true} if table is online. - * @throws IOException if a remote or network exception occurs - * @deprecated use {@link HBaseAdmin#isTableEnabled(byte[])} - */ - @Deprecated - public static boolean isTableEnabled(Configuration conf, String tableName) - throws IOException { - return isTableEnabled(conf, TableName.valueOf(tableName)); - } - - /** - * Tells whether or not a table is enabled or not. - * @param conf The Configuration object to use. - * @param tableName Name of table to check. - * @return {@code true} if table is online. - * @throws IOException if a remote or network exception occurs - * @deprecated use {@link HBaseAdmin#isTableEnabled(byte[])} - */ - @Deprecated - public static boolean isTableEnabled(Configuration conf, byte[] tableName) - throws IOException { - return isTableEnabled(conf, TableName.valueOf(tableName)); - } - - /** - * Tells whether or not a table is enabled or not. - * @param conf The Configuration object to use. - * @param tableName Name of table to check. - * @return {@code true} if table is online. - * @throws IOException if a remote or network exception occurs - * @deprecated use {@link HBaseAdmin#isTableEnabled(org.apache.hadoop.hbase.TableName tableName)} - */ - @Deprecated - public static boolean isTableEnabled(Configuration conf, - final TableName tableName) throws IOException { - try(Connection conn = ConnectionFactory.createConnection(conf)) { - return conn.getAdmin().isTableEnabled(tableName); - } - } - - /** - * Find region location hosting passed row using cached info - * @param row Row to find. - * @return The location of the given row. - * @throws IOException if a remote or network exception occurs - * @deprecated Use {@link RegionLocator#getRegionLocation(byte[])} - */ - @Deprecated - public HRegionLocation getRegionLocation(final String row) - throws IOException { - return getRegionLocation(Bytes.toBytes(row), false); - } - - /** - * @deprecated Use {@link RegionLocator#getRegionLocation(byte[])} instead. - */ - @Deprecated - public HRegionLocation getRegionLocation(final byte [] row) - throws IOException { - return locator.getRegionLocation(row); - } - - /** - * @deprecated Use {@link RegionLocator#getRegionLocation(byte[], boolean)} instead. - */ - @Deprecated - public HRegionLocation getRegionLocation(final byte [] row, boolean reload) - throws IOException { - return locator.getRegionLocation(row, reload); - } - - /** * {@inheritDoc} */ @Override @@ -384,15 +262,6 @@ public class HTable implements HTableInterface { } /** - * Kept in 0.96 for backward compatibility - * @deprecated since 0.96. This is an internal buffer that should not be read nor write. - */ - @Deprecated - public List getWriteBuffer() { - return mutator == null ? null : mutator.getWriteBuffer(); - } - - /** * {@inheritDoc} */ @Override @@ -433,93 +302,6 @@ public class HTable implements HTableInterface { } /** - * @deprecated Use {@link RegionLocator#getStartEndKeys()} instead; - */ - @Deprecated - public byte [][] getStartKeys() throws IOException { - return locator.getStartKeys(); - } - - /** - * @deprecated Use {@link RegionLocator#getEndKeys()} instead; - */ - @Deprecated - public byte[][] getEndKeys() throws IOException { - return locator.getEndKeys(); - } - - /** - * @deprecated Use {@link RegionLocator#getStartEndKeys()} instead; - */ - @Deprecated - public Pair getStartEndKeys() throws IOException { - return locator.getStartEndKeys(); - } - - /** - * Gets all the regions and their address for this table. - *

- * This is mainly useful for the MapReduce integration. - * @return A map of HRegionInfo with it's server address - * @throws IOException if a remote or network exception occurs - * @deprecated This is no longer a public API. Use {@link #getAllRegionLocations()} instead. - */ - @SuppressWarnings("deprecation") - @Deprecated - public NavigableMap getRegionLocations() throws IOException { - // TODO: Odd that this returns a Map of HRI to SN whereas getRegionLocator, singular, - // returns an HRegionLocation. - return MetaTableAccessor.allTableRegions(this.connection, getName()); - } - - /** - * Gets all the regions and their address for this table. - *

- * This is mainly useful for the MapReduce integration. - * @return A map of HRegionInfo with it's server address - * @throws IOException if a remote or network exception occurs - * - * @deprecated Use {@link RegionLocator#getAllRegionLocations()} instead; - */ - @Deprecated - public List getAllRegionLocations() throws IOException { - return locator.getAllRegionLocations(); - } - - /** - * Get the corresponding regions for an arbitrary range of keys. - *

- * @param startKey Starting row in range, inclusive - * @param endKey Ending row in range, exclusive - * @return A list of HRegionLocations corresponding to the regions that - * contain the specified range - * @throws IOException if a remote or network exception occurs - * @deprecated This is no longer a public API - */ - @Deprecated - public List getRegionsInRange(final byte [] startKey, - final byte [] endKey) throws IOException { - return getRegionsInRange(startKey, endKey, false); - } - - /** - * Get the corresponding regions for an arbitrary range of keys. - *

- * @param startKey Starting row in range, inclusive - * @param endKey Ending row in range, exclusive - * @param reload true to reload information or false to use cached information - * @return A list of HRegionLocations corresponding to the regions that - * contain the specified range - * @throws IOException if a remote or network exception occurs - * @deprecated This is no longer a public API - */ - @Deprecated - public List getRegionsInRange(final byte [] startKey, - final byte [] endKey, final boolean reload) throws IOException { - return getKeysAndRegionsInRange(startKey, endKey, false, reload).getSecond(); - } - - /** * Get the corresponding start keys and regions for an arbitrary range of * keys. *

@@ -529,9 +311,7 @@ public class HTable implements HTableInterface { * @return A pair of list of start keys and list of HRegionLocations that * contain the specified range * @throws IOException if a remote or network exception occurs - * @deprecated This is no longer a public API */ - @Deprecated private Pair, List> getKeysAndRegionsInRange( final byte[] startKey, final byte[] endKey, final boolean includeEndKey) throws IOException { @@ -549,9 +329,7 @@ public class HTable implements HTableInterface { * @return A pair of list of start keys and list of HRegionLocations that * contain the specified range * @throws IOException if a remote or network exception occurs - * @deprecated This is no longer a public API */ - @Deprecated private Pair, List> getKeysAndRegionsInRange( final byte[] startKey, final byte[] endKey, final boolean includeEndKey, final boolean reload) throws IOException { @@ -565,7 +343,7 @@ public class HTable implements HTableInterface { List regionsInRange = new ArrayList(); byte[] currentKey = startKey; do { - HRegionLocation regionLocation = getRegionLocation(currentKey, reload); + HRegionLocation regionLocation = getRegionLocator().getRegionLocation(currentKey, reload); keysInRange.add(currentKey); regionsInRange.add(regionLocation); currentKey = regionLocation.getRegionInfo().getEndKey(); @@ -577,35 +355,6 @@ public class HTable implements HTableInterface { } /** - * {@inheritDoc} - * @deprecated Use reversed scan instead. - */ - @Override - @Deprecated - public Result getRowOrBefore(final byte[] row, final byte[] family) - throws IOException { - RegionServerCallable callable = new RegionServerCallable(this.connection, - tableName, row) { - @Override - public Result call(int callTimeout) throws IOException { - PayloadCarryingRpcController controller = rpcControllerFactory.newController(); - controller.setPriority(tableName); - controller.setCallTimeout(callTimeout); - ClientProtos.GetRequest request = RequestConverter.buildGetRowOrBeforeRequest( - getLocation().getRegionInfo().getRegionName(), row, family); - try { - ClientProtos.GetResponse response = getStub().get(controller, request); - if (!response.hasResult()) return null; - return ProtobufUtil.toResult(response.getResult()); - } catch (ServiceException se) { - throw ProtobufUtil.getRemoteException(se); - } - } - }; - return rpcCallerFactory.newCaller().callWithRetries(callable, this.operationTimeout); - } - - /** * The underlying {@link HTable} must not be closed. * {@link HTableInterface#getScanner(Scan)} has other usage details. */ @@ -740,7 +489,8 @@ public class HTable implements HTableInterface { return new Result[]{get(gets.get(0))}; } try { - Object [] r1 = batch((List)gets); + Object[] r1 = new Object[gets.size()]; + batch((List) gets, r1); // translate. Result [] results = new Result[r1.length]; @@ -771,20 +521,6 @@ public class HTable implements HTableInterface { /** * {@inheritDoc} - * @deprecated If any exception is thrown by one of the actions, there is no way to - * retrieve the partially executed results. Use {@link #batch(List, Object[])} instead. - */ - @Deprecated - @Override - public Object[] batch(final List actions) - throws InterruptedException, IOException { - Object[] results = new Object[actions.size()]; - batch(actions, results); - return results; - } - - /** - * {@inheritDoc} */ @Override public void batchCallback( @@ -1221,9 +957,9 @@ public class HTable implements HTableInterface { exists.add(ge); } - Object[] r1; + Object[] r1= new Object[exists.size()]; try { - r1 = batch(exists); + batch(exists, r1); } catch (InterruptedException e) { throw (InterruptedIOException)new InterruptedIOException().initCause(e); } @@ -1241,21 +977,6 @@ public class HTable implements HTableInterface { /** * {@inheritDoc} - * @deprecated Use {@link #existsAll(java.util.List)} instead. - */ - @Override - @Deprecated - public Boolean[] exists(final List gets) throws IOException { - boolean[] results = existsAll(gets); - Boolean[] objectResults = new Boolean[results.length]; - for (int i = 0; i < results.length; ++i) { - objectResults[i] = results[i]; - } - return objectResults; - } - - /** - * {@inheritDoc} * @throws IOException */ @Override @@ -1353,19 +1074,6 @@ public class HTable implements HTableInterface { /** * {@inheritDoc} - * @deprecated in 0.96. When called with setAutoFlush(false), this function also - * set clearBufferOnFail to true, which is unexpected but kept for historical reasons. - * Replace it with setAutoFlush(false, false) if this is exactly what you want, or by - * {@link #setAutoFlushTo(boolean)} for all other cases. - */ - @Deprecated - @Override - public void setAutoFlush(boolean autoFlush) { - this.autoFlush = autoFlush; - } - - /** - * {@inheritDoc} */ @Override public void setAutoFlushTo(boolean autoFlush) { @@ -1419,101 +1127,6 @@ public class HTable implements HTableInterface { } /** - * Enable or disable region cache prefetch for the table. It will be - * applied for the given table's all HTable instances who share the same - * connection. By default, the cache prefetch is enabled. - * @param tableName name of table to configure. - * @param enable Set to true to enable region cache prefetch. Or set to - * false to disable it. - * @throws IOException - * @deprecated does nothing since 0.99 - */ - @Deprecated - public static void setRegionCachePrefetch(final byte[] tableName, - final boolean enable) throws IOException { - } - - /** - * @deprecated does nothing since 0.99 - */ - @Deprecated - public static void setRegionCachePrefetch( - final TableName tableName, - final boolean enable) throws IOException { - } - - /** - * Enable or disable region cache prefetch for the table. It will be - * applied for the given table's all HTable instances who share the same - * connection. By default, the cache prefetch is enabled. - * @param conf The Configuration object to use. - * @param tableName name of table to configure. - * @param enable Set to true to enable region cache prefetch. Or set to - * false to disable it. - * @throws IOException - * @deprecated does nothing since 0.99 - */ - @Deprecated - public static void setRegionCachePrefetch(final Configuration conf, - final byte[] tableName, final boolean enable) throws IOException { - } - - /** - * @deprecated does nothing since 0.99 - */ - @Deprecated - public static void setRegionCachePrefetch(final Configuration conf, - final TableName tableName, - final boolean enable) throws IOException { - } - - /** - * Check whether region cache prefetch is enabled or not for the table. - * @param conf The Configuration object to use. - * @param tableName name of table to check - * @return true if table's region cache prefecth is enabled. Otherwise - * it is disabled. - * @throws IOException - * @deprecated always return false since 0.99 - */ - @Deprecated - public static boolean getRegionCachePrefetch(final Configuration conf, - final byte[] tableName) throws IOException { - return false; - } - - /** - * @deprecated always return false since 0.99 - */ - @Deprecated - public static boolean getRegionCachePrefetch(final Configuration conf, - final TableName tableName) throws IOException { - return false; - } - - /** - * Check whether region cache prefetch is enabled or not for the table. - * @param tableName name of table to check - * @return true if table's region cache prefecth is enabled. Otherwise - * it is disabled. - * @throws IOException - * @deprecated always return false since 0.99 - */ - @Deprecated - public static boolean getRegionCachePrefetch(final byte[] tableName) throws IOException { - return false; - } - - /** - * @deprecated always return false since 0.99 - */ - @Deprecated - public static boolean getRegionCachePrefetch( - final TableName tableName) throws IOException { - return false; - } - - /** * Explicitly clears the region cache to fetch the latest value from META. * This is a power user function: avoid unless you know the ramifications. */ 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 index 745c770..4cd81e7 100644 --- 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 @@ -19,7 +19,6 @@ package org.apache.hadoop.hbase.client; import java.io.IOException; -import java.util.List; import org.apache.hadoop.hbase.classification.InterfaceAudience; import org.apache.hadoop.hbase.classification.InterfaceStability; @@ -46,26 +45,6 @@ public interface HTableInterface extends Table { byte[] getTableName(); /** - * @deprecated Use {@link #existsAll(java.util.List)} instead. - */ - @Deprecated - Boolean[] exists(List gets) throws IOException; - - - /** - * See {@link #setAutoFlush(boolean, boolean)} - * - * @param autoFlush - * Whether or not to enable 'auto-flush'. - * @deprecated in 0.96. When called with setAutoFlush(false), this function also - * set clearBufferOnFail to true, which is unexpected but kept for historical reasons. - * Replace it with setAutoFlush(false, false) if this is exactly what you want, though - * this is the method you want for most cases. - */ - @Deprecated - void setAutoFlush(boolean autoFlush); - - /** * Turns 'auto-flush' on or off. *

* When enabled (default), {@link Put} operations don't get buffered/delayed @@ -96,8 +75,7 @@ public interface HTableInterface extends Table { * 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. Use - * {@link #setAutoFlush(boolean)}} instead. + * @deprecated in 0.99 since setting clearBufferOnFail is deprecated. * @see BufferedMutator#flush() */ @Deprecated @@ -105,8 +83,8 @@ public interface HTableInterface extends Table { /** * Set the autoFlush behavior, without changing the value of {@code clearBufferOnFail}. - * @deprecated in 0.99 since setting clearBufferOnFail is deprecated. Use - * {@link #setAutoFlush(boolean)} instead, or better still, move on to {@link BufferedMutator} + * @deprecated in 0.99 since setting clearBufferOnFail is deprecated. Move on to + * {@link BufferedMutator} */ @Deprecated void setAutoFlushTo(boolean autoFlush); @@ -157,23 +135,4 @@ public interface HTableInterface extends Table { */ @Deprecated void setWriteBufferSize(long writeBufferSize) throws IOException; - - - /** - * Return the row that matches row exactly, - * or the one that immediately precedes it. - * - * @param row A row key. - * @param family Column family to include in the {@link Result}. - * @throws IOException if a remote or network exception occurs. - * @since 0.20.0 - * - * @deprecated As of version 0.92 this method is deprecated without - * replacement. Since version 0.96+, you can use reversed scan. - * getRowOrBefore is used internally to find entries in hbase:meta and makes - * various assumptions about the table (which are true for hbase:meta but not - * in general) to be efficient. - */ - @Deprecated - Result getRowOrBefore(byte[] row, byte[] family) 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 498c587..7fc4f89 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 @@ -120,21 +120,6 @@ public interface Table extends Closeable { InterruptedException; /** - * Same as {@link #batch(List, Object[])}, but returns an array of - * results instead of using a results parameter reference. - * - * @param actions list of Get, Put, Delete, Increment, Append objects - * @return the results from the actions. A null in the return array means that - * the call for that action failed, even after retries - * @throws IOException - * @since 0.90.0 - * @deprecated If any exception is thrown by one of the actions, there is no way to - * retrieve the partially executed results. Use {@link #batch(List, Object[])} instead. - */ - @Deprecated - Object[] batch(final List actions) throws IOException, InterruptedException; - - /** * Same as {@link #batch(List, Object[])}, but with a callback. * @since 0.96.0 */ @@ -144,7 +129,6 @@ public interface Table extends Closeable { throws IOException, InterruptedException; /** - * Same as {@link #batch(List)}, but with a callback. * * @since 0.96.0 * @deprecated If any exception is thrown by one of the actions, there is no way to retrieve the @@ -219,7 +203,7 @@ public interface Table extends Closeable { /** * Puts some data in the table. - * + * * @param put The data to put. * @throws IOException if a remote or network exception occurs. * @since 0.20.0 diff --git a/hbase-rest/src/main/java/org/apache/hadoop/hbase/rest/client/RemoteHTable.java b/hbase-rest/src/main/java/org/apache/hadoop/hbase/rest/client/RemoteHTable.java index 2d122df..0dc7f39 100644 --- a/hbase-rest/src/main/java/org/apache/hadoop/hbase/rest/client/RemoteHTable.java +++ b/hbase-rest/src/main/java/org/apache/hadoop/hbase/rest/client/RemoteHTable.java @@ -792,11 +792,6 @@ public class RemoteHTable implements Table { } @Override - public Object[] batch(List actions) throws IOException { - throw new IOException("batch not supported"); - } - - @Override public void batchCallback(List actions, Object[] results, Batch.Callback callback) throws IOException, InterruptedException { throw new IOException("batchCallback not supported"); 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 e3641c7..4805cdf 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 @@ -242,18 +242,6 @@ public final class HTableWrapper implements Table { table.batch(actions, results); } - /** - * {@inheritDoc} - * @deprecated If any exception is thrown by one of the actions, there is no way to - * retrieve the partially executed results. Use {@link #batch(List, Object[])} instead. - */ - @Deprecated - @Override - public Object[] batch(List actions) - throws IOException, InterruptedException { - return table.batch(actions); - } - @Override public void batchCallback(List actions, Object[] results, Batch.Callback callback) throws IOException, InterruptedException { diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/ReplicationSink.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/ReplicationSink.java index 7d47677..531c52b 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/ReplicationSink.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/ReplicationSink.java @@ -234,7 +234,7 @@ public class ReplicationSink { try { table = this.sharedHtableCon.getTable(tableName); for (List rows : allRows) { - table.batch(rows); + table.batch(rows, new Object[rows.size()]); } } catch (InterruptedException ix) { throw (InterruptedIOException)new InterruptedIOException().initCause(ix); 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 f057a44..45be52b 100644 --- a/hbase-server/src/main/resources/hbase-webapps/master/table.jsp +++ b/hbase-server/src/main/resources/hbase-webapps/master/table.jsp @@ -20,18 +20,21 @@ <%@ page contentType="text/html;charset=UTF-8" import="static org.apache.commons.lang.StringEscapeUtils.escapeXml" import="java.util.TreeMap" + import="java.util.List" import="java.util.Map" import="java.util.Set" import="java.util.Collection" import="org.apache.hadoop.conf.Configuration" import="org.apache.hadoop.hbase.client.HTable" import="org.apache.hadoop.hbase.client.Admin" + import="org.apache.hadoop.hbase.client.RegionLocator" import="org.apache.hadoop.hbase.HRegionInfo" + import="org.apache.hadoop.hbase.HRegionLocation" import="org.apache.hadoop.hbase.ServerName" import="org.apache.hadoop.hbase.ServerLoad" import="org.apache.hadoop.hbase.RegionLoad" import="org.apache.hadoop.hbase.HConstants" - import="org.apache.hadoop.hbase.master.HMaster" + import="org.apache.hadoop.hbase.master.HMaster" import="org.apache.hadoop.hbase.zookeeper.MetaTableLocator" import="org.apache.hadoop.hbase.util.Bytes" import="org.apache.hadoop.hbase.util.FSUtils" @@ -119,8 +122,8 @@ -<% -if ( fqtn != null ) { +<% +if ( fqtn != null ) { table = (HTable) master.getConnection().getTable(fqtn); if (table.getTableDescriptor().getRegionReplication() > 1) { tableHeader = "

Table Regions

"; @@ -128,7 +131,7 @@ if ( fqtn != null ) { } else { tableHeader = "

Table Regions

NameRegion ServerStart KeyEnd KeyLocalityRequestsReplicaID
"; } - if ( !readOnly && action != null ) { + if ( !readOnly && action != null ) { %>
@@ -147,7 +150,7 @@ if ( fqtn != null ) { } else { admin.split(TableName.valueOf(fqtn)); } - + %> Split request accepted. <% } else if (action.equals("compact")) { if (key != null && key.length() > 0) { @@ -201,6 +204,7 @@ if ( fqtn != null ) {
NameRegion ServerStart KeyEnd KeyLocalityRequests
<%} else { Admin admin = master.getConnection().getAdmin(); + RegionLocator r = master.getConnection().getRegionLocator(table.getName()); try { %>

Table Attributes

@@ -256,7 +260,7 @@ if ( fqtn != null ) {
- + <% Map familyValues = family.getValues(); @@ -278,13 +282,13 @@ if ( fqtn != null ) {
PropertyValueValue
<% Map regDistribution = new TreeMap(); - Map regions = table.getRegionLocations(); + List regions = r.getAllRegionLocations(); if(regions != null && regions.size() > 0) { %> <%= tableHeader %> <% - for (Map.Entry hriEntry : regions.entrySet()) { - HRegionInfo regionInfo = hriEntry.getKey(); - ServerName addr = hriEntry.getValue(); + for (HRegionLocation hriEntry : regions) { + HRegionInfo regionInfo = hriEntry.getRegionInfo(); + ServerName addr = hriEntry.getServerName(); long req = 0; float locality = 0.0f; String urlRegionServer = null; @@ -339,8 +343,8 @@ if ( fqtn != null ) {

Regions by Region Server

<% - for (Map.Entry rdEntry : regDistribution.entrySet()) { - ServerName addr = rdEntry.getKey(); + for (Map.Entry rdEntry : regDistribution.entrySet()) { + ServerName addr = rdEntry.getKey(); String url = "//" + addr.getHostname() + ":" + master.getRegionServerInfoPort(addr) + "/"; %> @@ -399,8 +403,8 @@ Actions: <% } %> -<% } -} else { // handle the case for fqtn is null with error message + redirect +<% } +} else { // handle the case for fqtn is null with error message + redirect %>
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAdmin1.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAdmin1.java index 02bc3c7..b00024e 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAdmin1.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAdmin1.java @@ -30,7 +30,6 @@ import java.util.HashMap; import java.util.Iterator; import java.util.List; import java.util.Map; -import java.util.Set; import java.util.concurrent.atomic.AtomicInteger; import org.apache.commons.logging.Log; @@ -39,6 +38,7 @@ 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.HRegionLocation; import org.apache.hadoop.hbase.HTableDescriptor; import org.apache.hadoop.hbase.InvalidFamilyOperationException; import org.apache.hadoop.hbase.MasterNotRunningException; @@ -626,27 +626,31 @@ public class TestAdmin1 { @SuppressWarnings("deprecation") protected void verifyRoundRobinDistribution(HTable ht, int expectedRegions) throws IOException { int numRS = ht.getConnection().getCurrentNrHRS(); - Map regions = ht.getRegionLocations(); - Map> server2Regions = new HashMap>(); - for (Map.Entry entry : regions.entrySet()) { - ServerName server = entry.getValue(); - List regs = server2Regions.get(server); - if (regs == null) { - regs = new ArrayList(); - server2Regions.put(server, regs); + try (RegionLocator r = TEST_UTIL.getConnection().getRegionLocator(ht.getName())) { + List regions = r.getAllRegionLocations(); + Map> server2Regions = + new HashMap>(); + for (HRegionLocation entry : regions) { + ServerName server = entry.getServerName(); + List regs = server2Regions.get(server); + if (regs == null) { + regs = new ArrayList(); + server2Regions.put(server, regs); + } + regs.add(entry.getRegionInfo()); + } + + if (numRS >= 2) { + // Ignore the master region server, + // which contains less regions by intention. + numRS--; + } + float average = (float) expectedRegions / numRS; + int min = (int) Math.floor(average); + int max = (int) Math.ceil(average); + for (List regionList : server2Regions.values()) { + assertTrue(regionList.size() == min || regionList.size() == max); } - regs.add(entry.getKey()); - } - if (numRS >= 2) { - // Ignore the master region server, - // which contains less regions by intention. - numRS--; - } - float average = (float) expectedRegions/numRS; - int min = (int)Math.floor(average); - int max = (int)Math.ceil(average); - for (List regionList : server2Regions.values()) { - assertTrue(regionList.size() == min || regionList.size() == max); } } @@ -656,28 +660,46 @@ public class TestAdmin1 { HTableDescriptor desc = new HTableDescriptor(tableName); desc.addFamily(new HColumnDescriptor(HConstants.CATALOG_FAMILY)); admin.createTable(desc); - HTable ht = (HTable) TEST_UTIL.getConnection().getTable(tableName); - Map regions = ht.getRegionLocations(); - assertEquals("Table should have only 1 region", 1, regions.size()); - ht.close(); + Connection c = TEST_UTIL.getConnection(); + RegionLocator r = null; + List regions = new ArrayList(); + try { + r = c.getRegionLocator(tableName); + regions = r.getAllRegionLocations(); + assertEquals("Table should have only 1 region", 1, regions.size()); + } finally { + if (r != null) { + r.close(); + } + } TableName TABLE_2 = TableName.valueOf(tableName.getNameAsString() + "_2"); desc = new HTableDescriptor(TABLE_2); desc.addFamily(new HColumnDescriptor(HConstants.CATALOG_FAMILY)); - admin.createTable(desc, new byte[][]{new byte[]{42}}); - HTable ht2 = (HTable) TEST_UTIL.getConnection().getTable(TABLE_2); - regions = ht2.getRegionLocations(); - assertEquals("Table should have only 2 region", 2, regions.size()); - ht2.close(); + admin.createTable(desc, new byte[][] { new byte[] { 42 } }); + try { + r = c.getRegionLocator(TABLE_2); + regions = r.getAllRegionLocations(); + assertEquals("Table should have only 2 region", 2, regions.size()); + } finally { + if (r != null) { + r.close(); + } + } TableName TABLE_3 = TableName.valueOf(tableName.getNameAsString() + "_3"); desc = new HTableDescriptor(TABLE_3); desc.addFamily(new HColumnDescriptor(HConstants.CATALOG_FAMILY)); admin.createTable(desc, "a".getBytes(), "z".getBytes(), 3); - HTable ht3 = (HTable) TEST_UTIL.getConnection().getTable(TABLE_3); - regions = ht3.getRegionLocations(); - assertEquals("Table should have only 3 region", 3, regions.size()); - ht3.close(); + try { + r = c.getRegionLocator(TABLE_3); + regions = r.getAllRegionLocations(); + assertEquals("Table should have only 3 region", 3, regions.size()); + } finally { + if (r != null) { + r.close(); + } + } TableName TABLE_4 = TableName.valueOf(tableName.getNameAsString() + "_4"); desc = new HTableDescriptor(TABLE_4); @@ -686,17 +708,22 @@ public class TestAdmin1 { admin.createTable(desc, "a".getBytes(), "z".getBytes(), 2); fail("Should not be able to create a table with only 2 regions using this API."); } catch (IllegalArgumentException eae) { - // Expected + // Expected } TableName TABLE_5 = TableName.valueOf(tableName.getNameAsString() + "_5"); desc = new HTableDescriptor(TABLE_5); desc.addFamily(new HColumnDescriptor(HConstants.CATALOG_FAMILY)); - admin.createTable(desc, new byte[] {1}, new byte[] {127}, 16); - HTable ht5 = (HTable) TEST_UTIL.getConnection().getTable(TABLE_5); - regions = ht5.getRegionLocations(); - assertEquals("Table should have 16 region", 16, regions.size()); - ht5.close(); + admin.createTable(desc, new byte[] { 1 }, new byte[] { 127 }, 16); + try { + r = c.getRegionLocator(TABLE_5); + regions = r.getAllRegionLocations(); + assertEquals("Table should have 16 region", 16, regions.size()); + } finally { + if (r != null) { + r.close(); + } + } } @Test (timeout=300000) @@ -724,151 +751,157 @@ public class TestAdmin1 { boolean tableAvailable = admin.isTableAvailable(tableName, splitKeys); assertTrue("Table should be created with splitKyes + 1 rows in META", tableAvailable); - HTable ht = (HTable) TEST_UTIL.getConnection().getTable(tableName); - Map regions = ht.getRegionLocations(); - assertEquals("Tried to create " + expectedRegions + " regions " + - "but only found " + regions.size(), + Connection connection = TEST_UTIL.getConnection(); + HTable ht = (HTable) connection.getTable(tableName); + RegionLocator r = null; + try { + r = connection.getRegionLocator(tableName); + List regions = r.getAllRegionLocations(); + assertEquals( + "Tried to create " + expectedRegions + " regions " + "but only found " + regions.size(), expectedRegions, regions.size()); - System.err.println("Found " + regions.size() + " regions"); - - Iterator hris = regions.keySet().iterator(); - HRegionInfo hri = hris.next(); - assertTrue(hri.getStartKey() == null || hri.getStartKey().length == 0); - assertTrue(Bytes.equals(hri.getEndKey(), splitKeys[0])); - hri = hris.next(); - assertTrue(Bytes.equals(hri.getStartKey(), splitKeys[0])); - assertTrue(Bytes.equals(hri.getEndKey(), splitKeys[1])); - hri = hris.next(); - assertTrue(Bytes.equals(hri.getStartKey(), splitKeys[1])); - assertTrue(Bytes.equals(hri.getEndKey(), splitKeys[2])); - hri = hris.next(); - assertTrue(Bytes.equals(hri.getStartKey(), splitKeys[2])); - assertTrue(Bytes.equals(hri.getEndKey(), splitKeys[3])); - hri = hris.next(); - assertTrue(Bytes.equals(hri.getStartKey(), splitKeys[3])); - assertTrue(Bytes.equals(hri.getEndKey(), splitKeys[4])); - hri = hris.next(); - assertTrue(Bytes.equals(hri.getStartKey(), splitKeys[4])); - assertTrue(Bytes.equals(hri.getEndKey(), splitKeys[5])); - hri = hris.next(); - assertTrue(Bytes.equals(hri.getStartKey(), splitKeys[5])); - assertTrue(Bytes.equals(hri.getEndKey(), splitKeys[6])); - hri = hris.next(); - assertTrue(Bytes.equals(hri.getStartKey(), splitKeys[6])); - assertTrue(Bytes.equals(hri.getEndKey(), splitKeys[7])); - hri = hris.next(); - assertTrue(Bytes.equals(hri.getStartKey(), splitKeys[7])); - assertTrue(Bytes.equals(hri.getEndKey(), splitKeys[8])); - hri = hris.next(); - assertTrue(Bytes.equals(hri.getStartKey(), splitKeys[8])); - assertTrue(hri.getEndKey() == null || hri.getEndKey().length == 0); - - verifyRoundRobinDistribution(ht, expectedRegions); - ht.close(); - - // Now test using start/end with a number of regions - - // Use 80 bit numbers to make sure we aren't limited - byte [] startKey = { 1, 1, 1, 1, 1, 1, 1, 1, 1, 1 }; - byte [] endKey = { 9, 9, 9, 9, 9, 9, 9, 9, 9, 9 }; - - // Splitting into 10 regions, we expect (null,1) ... (9, null) - // with (1,2) (2,3) (3,4) (4,5) (5,6) (6,7) (7,8) (8,9) in the middle - - expectedRegions = 10; - - TableName TABLE_2 = TableName.valueOf(tableName.getNameAsString() + "_2"); - - desc = new HTableDescriptor(TABLE_2); - desc.addFamily(new HColumnDescriptor(HConstants.CATALOG_FAMILY)); - admin = TEST_UTIL.getHBaseAdmin(); - admin.createTable(desc, startKey, endKey, expectedRegions); - - HTable ht2 = (HTable) TEST_UTIL.getConnection().getTable(TABLE_2); - regions = ht2.getRegionLocations(); - assertEquals("Tried to create " + expectedRegions + " regions " + - "but only found " + regions.size(), + System.err.println("Found " + regions.size() + " regions"); + + Iterator hris = regions.iterator(); + HRegionInfo hri = hris.next().getRegionInfo(); + assertTrue(hri.getStartKey() == null || hri.getStartKey().length == 0); + assertTrue(Bytes.equals(hri.getEndKey(), splitKeys[0])); + hri = hris.next().getRegionInfo(); + assertTrue(Bytes.equals(hri.getStartKey(), splitKeys[0])); + assertTrue(Bytes.equals(hri.getEndKey(), splitKeys[1])); + hri = hris.next().getRegionInfo(); + assertTrue(Bytes.equals(hri.getStartKey(), splitKeys[1])); + assertTrue(Bytes.equals(hri.getEndKey(), splitKeys[2])); + hri = hris.next().getRegionInfo(); + assertTrue(Bytes.equals(hri.getStartKey(), splitKeys[2])); + assertTrue(Bytes.equals(hri.getEndKey(), splitKeys[3])); + hri = hris.next().getRegionInfo(); + assertTrue(Bytes.equals(hri.getStartKey(), splitKeys[3])); + assertTrue(Bytes.equals(hri.getEndKey(), splitKeys[4])); + hri = hris.next().getRegionInfo(); + assertTrue(Bytes.equals(hri.getStartKey(), splitKeys[4])); + assertTrue(Bytes.equals(hri.getEndKey(), splitKeys[5])); + hri = hris.next().getRegionInfo(); + assertTrue(Bytes.equals(hri.getStartKey(), splitKeys[5])); + assertTrue(Bytes.equals(hri.getEndKey(), splitKeys[6])); + hri = hris.next().getRegionInfo(); + assertTrue(Bytes.equals(hri.getStartKey(), splitKeys[6])); + assertTrue(Bytes.equals(hri.getEndKey(), splitKeys[7])); + hri = hris.next().getRegionInfo(); + assertTrue(Bytes.equals(hri.getStartKey(), splitKeys[7])); + assertTrue(Bytes.equals(hri.getEndKey(), splitKeys[8])); + hri = hris.next().getRegionInfo(); + assertTrue(Bytes.equals(hri.getStartKey(), splitKeys[8])); + assertTrue(hri.getEndKey() == null || hri.getEndKey().length == 0); + + verifyRoundRobinDistribution(ht, expectedRegions); + ht.close(); + + // Now test using start/end with a number of regions + + // Use 80 bit numbers to make sure we aren't limited + byte[] startKey = { 1, 1, 1, 1, 1, 1, 1, 1, 1, 1 }; + byte[] endKey = { 9, 9, 9, 9, 9, 9, 9, 9, 9, 9 }; + + // Splitting into 10 regions, we expect (null,1) ... (9, null) + // with (1,2) (2,3) (3,4) (4,5) (5,6) (6,7) (7,8) (8,9) in the middle + + expectedRegions = 10; + + TableName TABLE_2 = TableName.valueOf(tableName.getNameAsString() + "_2"); + + desc = new HTableDescriptor(TABLE_2); + desc.addFamily(new HColumnDescriptor(HConstants.CATALOG_FAMILY)); + admin = TEST_UTIL.getHBaseAdmin(); + admin.createTable(desc, startKey, endKey, expectedRegions); + + HTable ht2 = (HTable) connection.getTable(TABLE_2); + r = connection.getRegionLocator(TABLE_2); + regions = r.getAllRegionLocations(); + assertEquals( + "Tried to create " + expectedRegions + " regions " + "but only found " + regions.size(), expectedRegions, regions.size()); - System.err.println("Found " + regions.size() + " regions"); - - hris = regions.keySet().iterator(); - hri = hris.next(); - assertTrue(hri.getStartKey() == null || hri.getStartKey().length == 0); - assertTrue(Bytes.equals(hri.getEndKey(), new byte [] {1,1,1,1,1,1,1,1,1,1})); - hri = hris.next(); - assertTrue(Bytes.equals(hri.getStartKey(), new byte [] {1,1,1,1,1,1,1,1,1,1})); - assertTrue(Bytes.equals(hri.getEndKey(), new byte [] {2,2,2,2,2,2,2,2,2,2})); - hri = hris.next(); - assertTrue(Bytes.equals(hri.getStartKey(), new byte [] {2,2,2,2,2,2,2,2,2,2})); - assertTrue(Bytes.equals(hri.getEndKey(), new byte [] {3,3,3,3,3,3,3,3,3,3})); - hri = hris.next(); - assertTrue(Bytes.equals(hri.getStartKey(), new byte [] {3,3,3,3,3,3,3,3,3,3})); - assertTrue(Bytes.equals(hri.getEndKey(), new byte [] {4,4,4,4,4,4,4,4,4,4})); - hri = hris.next(); - assertTrue(Bytes.equals(hri.getStartKey(), new byte [] {4,4,4,4,4,4,4,4,4,4})); - assertTrue(Bytes.equals(hri.getEndKey(), new byte [] {5,5,5,5,5,5,5,5,5,5})); - hri = hris.next(); - assertTrue(Bytes.equals(hri.getStartKey(), new byte [] {5,5,5,5,5,5,5,5,5,5})); - assertTrue(Bytes.equals(hri.getEndKey(), new byte [] {6,6,6,6,6,6,6,6,6,6})); - hri = hris.next(); - assertTrue(Bytes.equals(hri.getStartKey(), new byte [] {6,6,6,6,6,6,6,6,6,6})); - assertTrue(Bytes.equals(hri.getEndKey(), new byte [] {7,7,7,7,7,7,7,7,7,7})); - hri = hris.next(); - assertTrue(Bytes.equals(hri.getStartKey(), new byte [] {7,7,7,7,7,7,7,7,7,7})); - assertTrue(Bytes.equals(hri.getEndKey(), new byte [] {8,8,8,8,8,8,8,8,8,8})); - hri = hris.next(); - assertTrue(Bytes.equals(hri.getStartKey(), new byte [] {8,8,8,8,8,8,8,8,8,8})); - assertTrue(Bytes.equals(hri.getEndKey(), new byte [] {9,9,9,9,9,9,9,9,9,9})); - hri = hris.next(); - assertTrue(Bytes.equals(hri.getStartKey(), new byte [] {9,9,9,9,9,9,9,9,9,9})); - assertTrue(hri.getEndKey() == null || hri.getEndKey().length == 0); - - verifyRoundRobinDistribution(ht2, expectedRegions); - ht2.close(); - - // Try once more with something that divides into something infinite - - startKey = new byte [] { 0, 0, 0, 0, 0, 0 }; - endKey = new byte [] { 1, 0, 0, 0, 0, 0 }; - - expectedRegions = 5; - - TableName TABLE_3 = TableName.valueOf(tableName.getNameAsString() + "_3"); - - desc = new HTableDescriptor(TABLE_3); - desc.addFamily(new HColumnDescriptor(HConstants.CATALOG_FAMILY)); - admin = TEST_UTIL.getHBaseAdmin(); - admin.createTable(desc, startKey, endKey, expectedRegions); - - - HTable ht3 = (HTable) TEST_UTIL.getConnection().getTable(TABLE_3); - regions = ht3.getRegionLocations(); - assertEquals("Tried to create " + expectedRegions + " regions " + - "but only found " + regions.size(), + System.err.println("Found " + regions.size() + " regions"); + + hris = regions.iterator(); + hri = hris.next().getRegionInfo(); + assertTrue(hri.getStartKey() == null || hri.getStartKey().length == 0); + assertTrue(Bytes.equals(hri.getEndKey(), new byte[] { 1, 1, 1, 1, 1, 1, 1, 1, 1, 1 })); + hri = hris.next().getRegionInfo(); + assertTrue(Bytes.equals(hri.getStartKey(), new byte[] { 1, 1, 1, 1, 1, 1, 1, 1, 1, 1 })); + assertTrue(Bytes.equals(hri.getEndKey(), new byte[] { 2, 2, 2, 2, 2, 2, 2, 2, 2, 2 })); + hri = hris.next().getRegionInfo(); + assertTrue(Bytes.equals(hri.getStartKey(), new byte[] { 2, 2, 2, 2, 2, 2, 2, 2, 2, 2 })); + assertTrue(Bytes.equals(hri.getEndKey(), new byte[] { 3, 3, 3, 3, 3, 3, 3, 3, 3, 3 })); + hri = hris.next().getRegionInfo(); + assertTrue(Bytes.equals(hri.getStartKey(), new byte[] { 3, 3, 3, 3, 3, 3, 3, 3, 3, 3 })); + assertTrue(Bytes.equals(hri.getEndKey(), new byte[] { 4, 4, 4, 4, 4, 4, 4, 4, 4, 4 })); + hri = hris.next().getRegionInfo(); + assertTrue(Bytes.equals(hri.getStartKey(), new byte[] { 4, 4, 4, 4, 4, 4, 4, 4, 4, 4 })); + assertTrue(Bytes.equals(hri.getEndKey(), new byte[] { 5, 5, 5, 5, 5, 5, 5, 5, 5, 5 })); + hri = hris.next().getRegionInfo(); + assertTrue(Bytes.equals(hri.getStartKey(), new byte[] { 5, 5, 5, 5, 5, 5, 5, 5, 5, 5 })); + assertTrue(Bytes.equals(hri.getEndKey(), new byte[] { 6, 6, 6, 6, 6, 6, 6, 6, 6, 6 })); + hri = hris.next().getRegionInfo(); + assertTrue(Bytes.equals(hri.getStartKey(), new byte[] { 6, 6, 6, 6, 6, 6, 6, 6, 6, 6 })); + assertTrue(Bytes.equals(hri.getEndKey(), new byte[] { 7, 7, 7, 7, 7, 7, 7, 7, 7, 7 })); + hri = hris.next().getRegionInfo(); + assertTrue(Bytes.equals(hri.getStartKey(), new byte[] { 7, 7, 7, 7, 7, 7, 7, 7, 7, 7 })); + assertTrue(Bytes.equals(hri.getEndKey(), new byte[] { 8, 8, 8, 8, 8, 8, 8, 8, 8, 8 })); + hri = hris.next().getRegionInfo(); + assertTrue(Bytes.equals(hri.getStartKey(), new byte[] { 8, 8, 8, 8, 8, 8, 8, 8, 8, 8 })); + assertTrue(Bytes.equals(hri.getEndKey(), new byte[] { 9, 9, 9, 9, 9, 9, 9, 9, 9, 9 })); + hri = hris.next().getRegionInfo(); + assertTrue(Bytes.equals(hri.getStartKey(), new byte[] { 9, 9, 9, 9, 9, 9, 9, 9, 9, 9 })); + assertTrue(hri.getEndKey() == null || hri.getEndKey().length == 0); + + verifyRoundRobinDistribution(ht2, expectedRegions); + ht2.close(); + + // Try once more with something that divides into something infinite + + startKey = new byte[] { 0, 0, 0, 0, 0, 0 }; + endKey = new byte[] { 1, 0, 0, 0, 0, 0 }; + + expectedRegions = 5; + + TableName TABLE_3 = TableName.valueOf(tableName.getNameAsString() + "_3"); + + desc = new HTableDescriptor(TABLE_3); + desc.addFamily(new HColumnDescriptor(HConstants.CATALOG_FAMILY)); + admin = TEST_UTIL.getHBaseAdmin(); + admin.createTable(desc, startKey, endKey, expectedRegions); + + HTable ht3 = (HTable) connection.getTable(TABLE_3); + r = connection.getRegionLocator(TABLE_3); + regions = r.getAllRegionLocations(); + assertEquals( + "Tried to create " + expectedRegions + " regions " + "but only found " + regions.size(), expectedRegions, regions.size()); - System.err.println("Found " + regions.size() + " regions"); - - verifyRoundRobinDistribution(ht3, expectedRegions); - ht3.close(); + System.err.println("Found " + regions.size() + " regions"); + verifyRoundRobinDistribution(ht3, expectedRegions); + ht3.close(); - // Try an invalid case where there are duplicate split keys - splitKeys = new byte [][] { - new byte [] { 1, 1, 1 }, - new byte [] { 2, 2, 2 }, - new byte [] { 3, 3, 3 }, - new byte [] { 2, 2, 2 } - }; + // Try an invalid case where there are duplicate split keys + splitKeys = + new byte[][] { new byte[] { 1, 1, 1 }, new byte[] { 2, 2, 2 }, new byte[] { 3, 3, 3 }, + new byte[] { 2, 2, 2 } }; - TableName TABLE_4 = TableName.valueOf(tableName.getNameAsString() + "_4"); - desc = new HTableDescriptor(TABLE_4); - desc.addFamily(new HColumnDescriptor(HConstants.CATALOG_FAMILY)); - try { - admin.createTable(desc, splitKeys); - assertTrue("Should not be able to create this table because of " + - "duplicate split keys", false); - } catch(IllegalArgumentException iae) { - // Expected + TableName TABLE_4 = TableName.valueOf(tableName.getNameAsString() + "_4"); + desc = new HTableDescriptor(TABLE_4); + desc.addFamily(new HColumnDescriptor(HConstants.CATALOG_FAMILY)); + try { + admin.createTable(desc, splitKeys); + assertTrue("Should not be able to create this table because of " + "duplicate split keys", + false); + } catch (IllegalArgumentException iae) { + // Expected + } + } finally { + if (r != null) { + r.close(); + } } } @@ -962,20 +995,31 @@ public class TestAdmin1 { HTableDescriptor desc = new HTableDescriptor(tableName); desc.addFamily(new HColumnDescriptor(HConstants.CATALOG_FAMILY)); admin.createTable(desc, splitKeys); - HTable ht = (HTable) TEST_UTIL.getConnection().getTable(tableName); - Map regions = ht.getRegionLocations(); - assertEquals("Tried to create " + expectedRegions + " regions " - + "but only found " + regions.size(), expectedRegions, regions.size()); - // Disable table. - admin.disableTable(tableName); - // Enable table, use retain assignment to assign regions. - admin.enableTable(tableName); - Map regions2 = ht.getRegionLocations(); - - // Check the assignment. - assertEquals(regions.size(), regions2.size()); - for (Map.Entry entry : regions.entrySet()) { - assertEquals(regions2.get(entry.getKey()), entry.getValue()); + try (RegionLocator r = TEST_UTIL.getConnection().getRegionLocator(tableName)) { + List regions = r.getAllRegionLocations(); + assertEquals( + "Tried to create " + expectedRegions + " regions " + "but only found " + regions.size(), + expectedRegions, regions.size()); + // Disable table. + admin.disableTable(tableName); + // Enable table, use retain assignment to assign regions. + admin.enableTable(tableName); + List regions2 = r.getAllRegionLocations(); + + // Check the assignment. + assertEquals(regions.size(), regions2.size()); + for (HRegionLocation entry : regions) { + for (int i = 0; i < regions2.size(); i++) { + if (entry.getRegionInfo().equals(regions.get(i).getRegionInfo())) { + if (entry.getServerName().equals(regions.get(i).getServerName())) { + break; + } else { + fail("Found different server names " + entry.getServerName() + ", " + + regions.get(i).getServerName() + " for region info " + entry.getRegionInfo()); + } + } + } + } } } @@ -1044,105 +1088,107 @@ public class TestAdmin1 { rowCount = rowCounts[index]; } } - - // get the initial layout (should just be one region) - Map m = table.getRegionLocations(); - LOG.info("Initial regions (" + m.size() + "): " + m); - assertTrue(m.size() == 1); - - // Verify row count - Scan scan = new Scan(); - ResultScanner scanner = table.getScanner(scan); - int rows = 0; - for(@SuppressWarnings("unused") Result result : scanner) { - rows++; - } - scanner.close(); - assertEquals(rowCount, rows); - - // Have an outstanding scan going on to make sure we can scan over splits. - scan = new Scan(); - scanner = table.getScanner(scan); - // Scan first row so we are into first region before split happens. - scanner.next(); - - // Split the table - this.admin.split(tableName, splitPoint); - - final AtomicInteger count = new AtomicInteger(0); - Thread t = new Thread("CheckForSplit") { - @Override - public void run() { - for (int i = 0; i < 45; i++) { - try { - sleep(1000); - } catch (InterruptedException e) { - continue; - } - // check again table = new HTable(conf, tableName); - Map regions = null; - try { - regions = table.getRegionLocations(); - } catch (IOException e) { - e.printStackTrace(); - } - if (regions == null) continue; - count.set(regions.size()); - if (count.get() >= 2) { - LOG.info("Found: " + regions); - break; + try (RegionLocator rl = TEST_UTIL.getConnection().getRegionLocator(tableName)) { + // get the initial layout (should just be one region) + List m = rl.getAllRegionLocations(); + LOG.info("Initial regions (" + m.size() + "): " + m); + assertTrue(m.size() == 1); + + // Verify row count + Scan scan = new Scan(); + ResultScanner scanner = table.getScanner(scan); + int rows = 0; + for (@SuppressWarnings("unused") + Result result : scanner) { + rows++; + } + scanner.close(); + assertEquals(rowCount, rows); + + // Have an outstanding scan going on to make sure we can scan over splits. + scan = new Scan(); + scanner = table.getScanner(scan); + // Scan first row so we are into first region before split happens. + scanner.next(); + + // Split the table + this.admin.split(tableName, splitPoint); + + final AtomicInteger count = new AtomicInteger(0); + Thread t = new Thread("CheckForSplit") { + @Override + public void run() { + for (int i = 0; i < 45; i++) { + try { + sleep(1000); + } catch (InterruptedException e) { + continue; + } + // check again table = new HTable(conf, tableName); + List regions = null; + try { + regions = rl.getAllRegionLocations(); + } catch (IOException e) { + e.printStackTrace(); + } + if (regions == null) continue; + count.set(regions.size()); + if (count.get() >= 2) { + LOG.info("Found: " + regions); + break; + } + LOG.debug("Cycle waiting on split"); } - LOG.debug("Cycle waiting on split"); + LOG.debug("CheckForSplit thread exited, current region count: " + count.get()); + } + }; + t.setPriority(Thread.NORM_PRIORITY - 2); + t.start(); + t.join(); + + // Verify row count + rows = 1; // We counted one row above. + for (@SuppressWarnings("unused") + Result result : scanner) { + rows++; + if (rows > rowCount) { + scanner.close(); + assertTrue("Scanned more than expected (" + rowCount + ")", false); } - LOG.debug("CheckForSplit thread exited, current region count: " + count.get()); - } - }; - t.setPriority(Thread.NORM_PRIORITY - 2); - t.start(); - t.join(); - - // Verify row count - rows = 1; // We counted one row above. - for (@SuppressWarnings("unused") Result result : scanner) { - rows++; - if (rows > rowCount) { - scanner.close(); - assertTrue("Scanned more than expected (" + rowCount + ")", false); } - } - scanner.close(); - assertEquals(rowCount, rows); + scanner.close(); + assertEquals(rowCount, rows); - Map regions = null; - try { - regions = table.getRegionLocations(); - } catch (IOException e) { - e.printStackTrace(); - } - assertEquals(2, regions.size()); - Set hRegionInfos = regions.keySet(); - HRegionInfo[] r = hRegionInfos.toArray(new HRegionInfo[hRegionInfos.size()]); - if (splitPoint != null) { - // make sure the split point matches our explicit configuration - assertEquals(Bytes.toString(splitPoint), - Bytes.toString(r[0].getEndKey())); - assertEquals(Bytes.toString(splitPoint), - Bytes.toString(r[1].getStartKey())); - LOG.debug("Properly split on " + Bytes.toString(splitPoint)); - } else { - if (familyNames.length > 1) { - int splitKey = Bytes.toInt(r[0].getEndKey()); - // check if splitKey is based on the largest column family - // in terms of it store size - int deltaForLargestFamily = Math.abs(rowCount/2 - splitKey); - LOG.debug("SplitKey=" + splitKey + "&deltaForLargestFamily=" + deltaForLargestFamily + - ", r=" + r[0]); - for (int index = 0; index < familyNames.length; index++) { - int delta = Math.abs(rowCounts[index]/2 - splitKey); - if (delta < deltaForLargestFamily) { - assertTrue("Delta " + delta + " for family " + index - + " should be at least deltaForLargestFamily " + deltaForLargestFamily, - false); + List regions = null; + try { + regions = rl.getAllRegionLocations(); + } catch (IOException e) { + e.printStackTrace(); + } + assertEquals(2, regions.size()); + HRegionInfo[] r = new HRegionInfo[regions.size()]; + for (int i = 0; i < regions.size(); i++) { + r[i] = regions.get(i).getRegionInfo(); + } + if (splitPoint != null) { + // make sure the split point matches our explicit configuration + assertEquals(Bytes.toString(splitPoint), Bytes.toString(r[0].getEndKey())); + assertEquals(Bytes.toString(splitPoint), Bytes.toString(r[1].getStartKey())); + LOG.debug("Properly split on " + Bytes.toString(splitPoint)); + } else { + if (familyNames.length > 1) { + int splitKey = Bytes.toInt(r[0].getEndKey()); + // check if splitKey is based on the largest column family + // in terms of it store size + int deltaForLargestFamily = Math.abs(rowCount / 2 - splitKey); + LOG.debug("SplitKey=" + splitKey + "&deltaForLargestFamily=" + deltaForLargestFamily + + ", r=" + r[0]); + for (int index = 0; index < familyNames.length; index++) { + int delta = Math.abs(rowCounts[index] / 2 - splitKey); + if (delta < deltaForLargestFamily) { + assertTrue("Delta " + delta + " for family " + index + + " should be at least deltaForLargestFamily " + deltaForLargestFamily, false); + } } } } diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAdmin2.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAdmin2.java index b121c37..18c7405 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAdmin2.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAdmin2.java @@ -50,7 +50,6 @@ import org.apache.hadoop.hbase.constraint.ConstraintException; import org.apache.hadoop.hbase.master.AssignmentManager; import org.apache.hadoop.hbase.master.HMaster; import org.apache.hadoop.hbase.protobuf.ProtobufUtil; -import org.apache.hadoop.hbase.regionserver.HRegion; import org.apache.hadoop.hbase.regionserver.HRegionServer; import org.apache.hadoop.hbase.regionserver.Region; import org.apache.hadoop.hbase.testclassification.ClientTests; @@ -685,7 +684,7 @@ public class TestAdmin2 { } } - @Test (timeout=300000) + @Test(timeout = 300000) public void testGetRegion() throws Exception { // We use actual HBaseAdmin instance instead of going via Admin interface in // here because makes use of an internal HBA method (TODO: Fix.). @@ -693,15 +692,18 @@ public class TestAdmin2 { final TableName tableName = TableName.valueOf("testGetRegion"); LOG.info("Started " + tableName); - HTable t = TEST_UTIL.createMultiRegionTable(tableName, HConstants.CATALOG_FAMILY); - - HRegionLocation regionLocation = t.getRegionLocation("mmm"); - HRegionInfo region = regionLocation.getRegionInfo(); - byte[] regionName = region.getRegionName(); - Pair pair = rawAdmin.getRegion(regionName); - assertTrue(Bytes.equals(regionName, pair.getFirst().getRegionName())); - pair = rawAdmin.getRegion(region.getEncodedNameAsBytes()); - assertTrue(Bytes.equals(regionName, pair.getFirst().getRegionName())); + TEST_UTIL.createMultiRegionTable(tableName, HConstants.CATALOG_FAMILY); + + try (Connection conn = ConnectionFactory.createConnection(TEST_UTIL.getConfiguration()); + RegionLocator r = conn.getRegionLocator(tableName);) { + HRegionLocation regionLocation = r.getRegionLocation(Bytes.toBytes("mmm")); + HRegionInfo region = regionLocation.getRegionInfo(); + byte[] regionName = region.getRegionName(); + Pair pair = rawAdmin.getRegion(regionName); + assertTrue(Bytes.equals(regionName, pair.getFirst().getRegionName())); + pair = rawAdmin.getRegion(region.getEncodedNameAsBytes()); + assertTrue(Bytes.equals(regionName, pair.getFirst().getRegionName())); + } } @Test(timeout = 30000) 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 027a348..03017f6 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 @@ -28,7 +28,6 @@ import static org.junit.Assert.assertTrue; import static org.junit.Assert.fail; import java.io.IOException; -import java.lang.reflect.Method; import java.util.ArrayList; import java.util.Arrays; import java.util.Collections; @@ -43,12 +42,10 @@ import java.util.concurrent.ExecutorService; import java.util.concurrent.Executors; import java.util.concurrent.atomic.AtomicReference; -import org.apache.log4j.Level; import org.apache.commons.lang.ArrayUtils; import org.apache.commons.logging.Log; import org.apache.commons.logging.LogFactory; import org.apache.hadoop.conf.Configuration; -import org.apache.hadoop.hbase.Abortable; import org.apache.hadoop.hbase.Cell; import org.apache.hadoop.hbase.CellUtil; import org.apache.hadoop.hbase.DoNotRetryIOException; @@ -93,7 +90,6 @@ import org.apache.hadoop.hbase.protobuf.generated.ClientProtos.MutationProto; import org.apache.hadoop.hbase.protobuf.generated.ClientProtos.MutationProto.MutationType; import org.apache.hadoop.hbase.protobuf.generated.MultiRowMutationProtos.MultiRowMutationService; import org.apache.hadoop.hbase.protobuf.generated.MultiRowMutationProtos.MutateRowsRequest; -import org.apache.hadoop.hbase.regionserver.HRegion; import org.apache.hadoop.hbase.regionserver.HRegionServer; import org.apache.hadoop.hbase.regionserver.NoSuchColumnFamilyException; import org.apache.hadoop.hbase.regionserver.Region; @@ -103,8 +99,8 @@ import org.apache.hadoop.hbase.testclassification.LargeTests; import org.apache.hadoop.hbase.util.Bytes; import org.apache.hadoop.hbase.util.EnvironmentEdgeManager; import org.apache.hadoop.hbase.util.Pair; -import org.apache.hadoop.hbase.zookeeper.ZooKeeperWatcher; import org.apache.log4j.AppenderSkeleton; +import org.apache.log4j.Level; import org.apache.log4j.Logger; import org.apache.log4j.spi.LoggingEvent; import org.junit.After; @@ -449,10 +445,10 @@ public class TestFromClientSide { int rowCount = TEST_UTIL.loadTable(t, FAMILY, false); assertRowCount(t, rowCount); // Split the table. Should split on a reasonable key; 'lqj' - Map regions = splitTable(t); + List regions = splitTable(t); assertRowCount(t, rowCount); // Get end key of first region. - byte [] endKey = regions.keySet().iterator().next().getEndKey(); + byte [] endKey = regions.iterator().next().getRegionInfo().getEndKey(); // Count rows with a filter that stops us before passed 'endKey'. // Should be count of rows in first region. int endKeyCount = countRows(t, createScanWithRowFilter(endKey)); @@ -548,16 +544,16 @@ public class TestFromClientSide { /* * Split table into multiple regions. * @param t Table to split. - * @return Map of regions to servers. + * @return List of table region locations. * @throws IOException */ - private Map splitTable(final HTable t) + private List splitTable(final HTable t) throws IOException, InterruptedException { // Split this table in two. HBaseAdmin admin = TEST_UTIL.getHBaseAdmin(); admin.split(t.getName()); admin.close(); - Map regions = waitOnSplit(t); + List regions = waitOnSplit(t); assertTrue(regions.size() > 1); return regions; } @@ -566,21 +562,23 @@ public class TestFromClientSide { * Wait on table split. May return because we waited long enough on the split * and it didn't happen. Caller should check. * @param t - * @return Map of table regions; caller needs to check table actually split. + * @return List of table regions locations; caller needs to check table actually split. */ - private Map waitOnSplit(final HTable t) - throws IOException { - Map regions = t.getRegionLocations(); - int originalCount = regions.size(); - for (int i = 0; i < TEST_UTIL.getConfiguration().getInt("hbase.test.retries", 30); i++) { - Thread.currentThread(); - try { - Thread.sleep(1000); - } catch (InterruptedException e) { - e.printStackTrace(); + private List waitOnSplit(final HTable t) throws IOException { + List regions; + try (RegionLocator r = TEST_UTIL.getConnection().getRegionLocator(t.getName())) { + regions = r.getAllRegionLocations(); + int originalCount = regions.size(); + for (int i = 0; i < TEST_UTIL.getConfiguration().getInt("hbase.test.retries", 30); i++) { + Thread.currentThread(); + try { + Thread.sleep(1000); + } catch (InterruptedException e) { + e.printStackTrace(); + } + regions = r.getAllRegionLocations(); + if (regions.size() > originalCount) break; } - regions = t.getRegionLocations(); - if (regions.size() > originalCount) break; } return regions; } @@ -3818,7 +3816,8 @@ public class TestFromClientSide { final int NB_BATCH_ROWS = 10; HTable table = TEST_UTIL.createTable(Bytes.toBytes("testRowsPutBufferedOneFlush"), new byte [][] {CONTENTS_FAMILY, SMALL_FAMILY}); - table.setAutoFlush(false); + table.setAutoFlushTo(false); + table.getBufferedMutator().flush(); ArrayList rowsUpdate = new ArrayList(); for (int i = 0; i < NB_BATCH_ROWS * 10; i++) { byte[] row = Bytes.toBytes("row" + i); @@ -4187,11 +4186,6 @@ public class TestFromClientSide { HTable table = TEST_UTIL.createTable(tableAname, new byte[][] { HConstants.CATALOG_FAMILY, Bytes.toBytes("info2") }, 1, 1024); - // set block size to 64 to making 2 kvs into one block, bypassing the walkForwardInSingleRow - // in Store.rowAtOrBeforeFromStoreFile - String regionName = table.getRegionLocations().firstKey().getEncodedName(); - Region region = - TEST_UTIL.getRSForFirstRegionInTable(tableAname).getFromOnlineRegions(regionName); Put put1 = new Put(firstRow); Put put2 = new Put(secondRow); Put put3 = new Put(thirdRow); @@ -4209,57 +4203,93 @@ public class TestFromClientSide { table.put(put2); table.put(put3); table.put(put4); - region.flush(true); + // set block size to 64 to making 2 kvs into one block, bypassing the walkForwardInSingleRow + // in Store.rowAtOrBeforeFromStoreFile + try (RegionLocator r = TEST_UTIL.getConnection().getRegionLocator(tableAname)) { + String regionName = r.getAllRegionLocations().get(0).getRegionInfo().getEncodedName(); + Region region = + TEST_UTIL.getRSForFirstRegionInTable(tableAname).getFromOnlineRegions(regionName); + region.flush(true); + } Result result = null; + Get get = null; // Test before first that null is returned - result = table.getRowOrBefore(beforeFirstRow, HConstants.CATALOG_FAMILY); - assertTrue(result == null); + get = new Get(beforeFirstRow); + get.setClosestRowBefore(true); + get.addFamily(HConstants.CATALOG_FAMILY); + + result = table.get(get); + assertTrue(result.isEmpty()); // Test at first that first is returned - result = table.getRowOrBefore(firstRow, HConstants.CATALOG_FAMILY); + get = new Get(firstRow); + get.setClosestRowBefore(true); + get.addFamily(HConstants.CATALOG_FAMILY); + result = table.get(get); 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 = table.getRowOrBefore(beforeSecondRow, HConstants.CATALOG_FAMILY); + get = new Get(beforeSecondRow); + get.setClosestRowBefore(true); + get.addFamily(HConstants.CATALOG_FAMILY); + result = table.get(get); 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 = table.getRowOrBefore(secondRow, HConstants.CATALOG_FAMILY); + get = new Get(secondRow); + get.setClosestRowBefore(true); + get.addFamily(HConstants.CATALOG_FAMILY); + result = table.get(get); 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 = table.getRowOrBefore(beforeThirdRow, HConstants.CATALOG_FAMILY); + get = new Get(beforeThirdRow); + get.setClosestRowBefore(true); + get.addFamily(HConstants.CATALOG_FAMILY); + result = table.get(get); 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 = table.getRowOrBefore(thirdRow, HConstants.CATALOG_FAMILY); + get = new Get(thirdRow); + get.setClosestRowBefore(true); + get.addFamily(HConstants.CATALOG_FAMILY); + result = table.get(get); 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 = table.getRowOrBefore(beforeForthRow, HConstants.CATALOG_FAMILY); + get = new Get(beforeForthRow); + get.setClosestRowBefore(true); + get.addFamily(HConstants.CATALOG_FAMILY); + result = table.get(get); 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 = table.getRowOrBefore(forthRow, HConstants.CATALOG_FAMILY); + get = new Get(forthRow); + get.setClosestRowBefore(true); + get.addFamily(HConstants.CATALOG_FAMILY); + result = table.get(get); 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 = table.getRowOrBefore(Bytes.add(forthRow, one), HConstants.CATALOG_FAMILY); + get = new Get(Bytes.add(forthRow, one)); + get.setClosestRowBefore(true); + get.addFamily(HConstants.CATALOG_FAMILY); + result = table.get(get); assertTrue(result.containsColumn(HConstants.CATALOG_FAMILY, null)); assertTrue(Bytes.equals(result.getRow(), forthRow)); assertTrue(Bytes.equals(result.getValue(HConstants.CATALOG_FAMILY, null), four)); @@ -4571,7 +4601,7 @@ public class TestFromClientSide { assertIncrementKey(kvs[0], ROW, FAMILY, QUALIFIERS[0], 2); assertIncrementKey(kvs[1], ROW, FAMILY, QUALIFIERS[1], 2); assertIncrementKey(kvs[2], ROW, FAMILY, QUALIFIERS[2], 2); - + ht.close(); } @@ -5093,96 +5123,98 @@ public class TestFromClientSide { byte [] data = Bytes.toBytes("data"); HTable table = TEST_UTIL.createTable(tableName, FAMILY); // get the block cache and region - String regionName = table.getRegionLocations().firstKey().getEncodedName(); - Region region = TEST_UTIL.getRSForFirstRegionInTable(tableName) - .getFromOnlineRegions(regionName); - Store store = region.getStores().iterator().next(); - CacheConfig cacheConf = store.getCacheConfig(); - cacheConf.setCacheDataOnWrite(true); - cacheConf.setEvictOnClose(true); - BlockCache cache = cacheConf.getBlockCache(); - - // establish baseline stats - long startBlockCount = cache.getBlockCount(); - long startBlockHits = cache.getStats().getHitCount(); - long startBlockMiss = cache.getStats().getMissCount(); - - // wait till baseline is stable, (minimal 500 ms) - for (int i = 0; i < 5; i++) { - Thread.sleep(100); - if (startBlockCount != cache.getBlockCount() - || startBlockHits != cache.getStats().getHitCount() - || startBlockMiss != cache.getStats().getMissCount()) { - startBlockCount = cache.getBlockCount(); - startBlockHits = cache.getStats().getHitCount(); - startBlockMiss = cache.getStats().getMissCount(); - i = -1; + try (RegionLocator rl = TEST_UTIL.getConnection().getRegionLocator(tableName)) { + String regionName = rl.getAllRegionLocations().get(0).getRegionInfo().getEncodedName(); + Region region = + TEST_UTIL.getRSForFirstRegionInTable(tableName).getFromOnlineRegions(regionName); + Store store = region.getStores().iterator().next(); + CacheConfig cacheConf = store.getCacheConfig(); + cacheConf.setCacheDataOnWrite(true); + cacheConf.setEvictOnClose(true); + BlockCache cache = cacheConf.getBlockCache(); + + // establish baseline stats + long startBlockCount = cache.getBlockCount(); + long startBlockHits = cache.getStats().getHitCount(); + long startBlockMiss = cache.getStats().getMissCount(); + + // wait till baseline is stable, (minimal 500 ms) + for (int i = 0; i < 5; i++) { + Thread.sleep(100); + if (startBlockCount != cache.getBlockCount() + || startBlockHits != cache.getStats().getHitCount() + || startBlockMiss != cache.getStats().getMissCount()) { + startBlockCount = cache.getBlockCount(); + startBlockHits = cache.getStats().getHitCount(); + startBlockMiss = cache.getStats().getMissCount(); + i = -1; + } } - } - // insert data - Put put = new Put(ROW); - put.add(FAMILY, QUALIFIER, data); - table.put(put); - assertTrue(Bytes.equals(table.get(new Get(ROW)).value(), data)); - // data was in memstore so don't expect any changes - assertEquals(startBlockCount, cache.getBlockCount()); - assertEquals(startBlockHits, cache.getStats().getHitCount()); - assertEquals(startBlockMiss, cache.getStats().getMissCount()); - // flush the data - System.out.println("Flushing cache"); - region.flush(true); - // expect one more block in cache, no change in hits/misses - long expectedBlockCount = startBlockCount + 1; - long expectedBlockHits = startBlockHits; - long expectedBlockMiss = startBlockMiss; - assertEquals(expectedBlockCount, cache.getBlockCount()); - assertEquals(expectedBlockHits, cache.getStats().getHitCount()); - assertEquals(expectedBlockMiss, cache.getStats().getMissCount()); - // read the data and expect same blocks, one new hit, no misses - assertTrue(Bytes.equals(table.get(new Get(ROW)).value(), data)); - assertEquals(expectedBlockCount, cache.getBlockCount()); - assertEquals(++expectedBlockHits, cache.getStats().getHitCount()); - assertEquals(expectedBlockMiss, cache.getStats().getMissCount()); - // insert a second column, read the row, no new blocks, one new hit - byte [] QUALIFIER2 = Bytes.add(QUALIFIER, QUALIFIER); - byte [] data2 = Bytes.add(data, data); - put = new Put(ROW); - put.add(FAMILY, QUALIFIER2, data2); - table.put(put); - Result r = table.get(new Get(ROW)); - assertTrue(Bytes.equals(r.getValue(FAMILY, QUALIFIER), data)); - assertTrue(Bytes.equals(r.getValue(FAMILY, QUALIFIER2), data2)); - assertEquals(expectedBlockCount, cache.getBlockCount()); - assertEquals(++expectedBlockHits, cache.getStats().getHitCount()); - assertEquals(expectedBlockMiss, cache.getStats().getMissCount()); - // flush, one new block - System.out.println("Flushing cache"); - region.flush(true); - assertEquals(++expectedBlockCount, cache.getBlockCount()); - assertEquals(expectedBlockHits, cache.getStats().getHitCount()); - assertEquals(expectedBlockMiss, cache.getStats().getMissCount()); - // compact, net minus two blocks, two hits, no misses - System.out.println("Compacting"); - assertEquals(2, store.getStorefilesCount()); - store.triggerMajorCompaction(); - region.compact(true); - waitForStoreFileCount(store, 1, 10000); // wait 10 seconds max - assertEquals(1, store.getStorefilesCount()); - expectedBlockCount -= 2; // evicted two blocks, cached none - assertEquals(expectedBlockCount, cache.getBlockCount()); - expectedBlockHits += 2; - assertEquals(expectedBlockMiss, cache.getStats().getMissCount()); - assertEquals(expectedBlockHits, cache.getStats().getHitCount()); - // read the row, this should be a cache miss because we don't cache data - // blocks on compaction - r = table.get(new Get(ROW)); - assertTrue(Bytes.equals(r.getValue(FAMILY, QUALIFIER), data)); - assertTrue(Bytes.equals(r.getValue(FAMILY, QUALIFIER2), data2)); - expectedBlockCount += 1; // cached one data block - assertEquals(expectedBlockCount, cache.getBlockCount()); - assertEquals(expectedBlockHits, cache.getStats().getHitCount()); - assertEquals(++expectedBlockMiss, cache.getStats().getMissCount()); + // insert data + Put put = new Put(ROW); + put.add(FAMILY, QUALIFIER, data); + table.put(put); + assertTrue(Bytes.equals(table.get(new Get(ROW)).value(), data)); + // data was in memstore so don't expect any changes + assertEquals(startBlockCount, cache.getBlockCount()); + assertEquals(startBlockHits, cache.getStats().getHitCount()); + assertEquals(startBlockMiss, cache.getStats().getMissCount()); + // flush the data + System.out.println("Flushing cache"); + region.flush(true); + // expect one more block in cache, no change in hits/misses + long expectedBlockCount = startBlockCount + 1; + long expectedBlockHits = startBlockHits; + long expectedBlockMiss = startBlockMiss; + assertEquals(expectedBlockCount, cache.getBlockCount()); + assertEquals(expectedBlockHits, cache.getStats().getHitCount()); + assertEquals(expectedBlockMiss, cache.getStats().getMissCount()); + // read the data and expect same blocks, one new hit, no misses + assertTrue(Bytes.equals(table.get(new Get(ROW)).value(), data)); + assertEquals(expectedBlockCount, cache.getBlockCount()); + assertEquals(++expectedBlockHits, cache.getStats().getHitCount()); + assertEquals(expectedBlockMiss, cache.getStats().getMissCount()); + // insert a second column, read the row, no new blocks, one new hit + byte[] QUALIFIER2 = Bytes.add(QUALIFIER, QUALIFIER); + byte[] data2 = Bytes.add(data, data); + put = new Put(ROW); + put.add(FAMILY, QUALIFIER2, data2); + table.put(put); + Result r = table.get(new Get(ROW)); + assertTrue(Bytes.equals(r.getValue(FAMILY, QUALIFIER), data)); + assertTrue(Bytes.equals(r.getValue(FAMILY, QUALIFIER2), data2)); + assertEquals(expectedBlockCount, cache.getBlockCount()); + assertEquals(++expectedBlockHits, cache.getStats().getHitCount()); + assertEquals(expectedBlockMiss, cache.getStats().getMissCount()); + // flush, one new block + System.out.println("Flushing cache"); + region.flush(true); + assertEquals(++expectedBlockCount, cache.getBlockCount()); + assertEquals(expectedBlockHits, cache.getStats().getHitCount()); + assertEquals(expectedBlockMiss, cache.getStats().getMissCount()); + // compact, net minus two blocks, two hits, no misses + System.out.println("Compacting"); + assertEquals(2, store.getStorefilesCount()); + store.triggerMajorCompaction(); + region.compact(true); + waitForStoreFileCount(store, 1, 10000); // wait 10 seconds max + assertEquals(1, store.getStorefilesCount()); + expectedBlockCount -= 2; // evicted two blocks, cached none + assertEquals(expectedBlockCount, cache.getBlockCount()); + expectedBlockHits += 2; + assertEquals(expectedBlockMiss, cache.getStats().getMissCount()); + assertEquals(expectedBlockHits, cache.getStats().getHitCount()); + // read the row, this should be a cache miss because we don't cache data + // blocks on compaction + r = table.get(new Get(ROW)); + assertTrue(Bytes.equals(r.getValue(FAMILY, QUALIFIER), data)); + assertTrue(Bytes.equals(r.getValue(FAMILY, QUALIFIER2), data2)); + expectedBlockCount += 1; // cached one data block + assertEquals(expectedBlockCount, cache.getBlockCount()); + assertEquals(expectedBlockHits, cache.getStats().getHitCount()); + assertEquals(++expectedBlockMiss, cache.getStats().getMissCount()); + } } private void waitForStoreFileCount(Store store, int count, int timeout) @@ -5206,15 +5238,17 @@ public class TestFromClientSide { TableName TABLE = TableName.valueOf("testNonCachedGetRegionLocation"); byte [] family1 = Bytes.toBytes("f1"); byte [] family2 = Bytes.toBytes("f2"); - try (HTable table = TEST_UTIL.createTable(TABLE, new byte[][] {family1, family2}, 10); - Admin admin = TEST_UTIL.getHBaseAdmin()) { - Map regionsMap = table.getRegionLocations(); + try (HTable table = TEST_UTIL.createTable(TABLE, new byte[][] { family1, family2 }, 10); + Admin admin = TEST_UTIL.getHBaseAdmin(); + RegionLocator r = TEST_UTIL.getConnection().getRegionLocator(TABLE);) { + List regionsMap = r.getAllRegionLocations(); assertEquals(1, regionsMap.size()); - HRegionInfo regionInfo = regionsMap.keySet().iterator().next(); - ServerName addrBefore = regionsMap.get(regionInfo); + HRegionLocation hrl = regionsMap.iterator().next(); + HRegionInfo regionInfo = hrl.getRegionInfo(); + ServerName addrBefore = hrl.getServerName(); // Verify region location before move. - HRegionLocation addrCache = table.getRegionLocation(regionInfo.getStartKey(), false); - HRegionLocation addrNoCache = table.getRegionLocation(regionInfo.getStartKey(), true); + HRegionLocation addrCache = r.getRegionLocation(regionInfo.getStartKey(), false); + HRegionLocation addrNoCache = r.getRegionLocation(regionInfo.getStartKey(), true); assertEquals(addrBefore.getPort(), addrCache.getPort()); assertEquals(addrBefore.getPort(), addrNoCache.getPort()); @@ -5235,8 +5269,8 @@ public class TestFromClientSide { } // Verify the region was moved. - addrCache = table.getRegionLocation(regionInfo.getStartKey(), false); - addrNoCache = table.getRegionLocation(regionInfo.getStartKey(), true); + addrCache = r.getRegionLocation(regionInfo.getStartKey(), false); + addrNoCache = r.getRegionLocation(regionInfo.getStartKey(), true); assertNotNull(addrAfter); assertTrue(addrAfter.getPort() != addrCache.getPort()); assertEquals(addrAfter.getPort(), addrNoCache.getPort()); @@ -5261,49 +5295,63 @@ public class TestFromClientSide { assertEquals(26, numOfRegions); // Get the regions in this range - List regionsList = table.getRegionsInRange(startKey, - endKey); + List regionsList = getRegionsInRange(TABLE, startKey, endKey); assertEquals(10, regionsList.size()); // Change the start key startKey = Bytes.toBytes("fff"); - regionsList = table.getRegionsInRange(startKey, endKey); + regionsList = getRegionsInRange(TABLE, startKey, endKey); assertEquals(7, regionsList.size()); // Change the end key endKey = Bytes.toBytes("nnn"); - regionsList = table.getRegionsInRange(startKey, endKey); + regionsList = getRegionsInRange(TABLE, startKey, endKey); assertEquals(8, regionsList.size()); // Empty start key - regionsList = table.getRegionsInRange(HConstants.EMPTY_START_ROW, endKey); + regionsList = getRegionsInRange(TABLE, HConstants.EMPTY_START_ROW, endKey); assertEquals(13, regionsList.size()); // Empty end key - regionsList = table.getRegionsInRange(startKey, HConstants.EMPTY_END_ROW); + regionsList = getRegionsInRange(TABLE, startKey, HConstants.EMPTY_END_ROW); assertEquals(21, regionsList.size()); // Both start and end keys empty - regionsList = table.getRegionsInRange(HConstants.EMPTY_START_ROW, - HConstants.EMPTY_END_ROW); + 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 = table.getRegionsInRange(startKey, endKey); + regionsList = getRegionsInRange(TABLE, startKey, endKey); assertEquals(21, regionsList.size()); // Change the start key to somewhere in the first block startKey = Bytes.toBytes("aac"); - regionsList = table.getRegionsInRange(startKey, endKey); + regionsList = getRegionsInRange(TABLE, startKey, endKey); assertEquals(26, regionsList.size()); // Make start and end key the same startKey = endKey = Bytes.toBytes("ccc"); - regionsList = table.getRegionsInRange(startKey, endKey); + regionsList = getRegionsInRange(TABLE, startKey, endKey); assertEquals(1, regionsList.size()); } + private List getRegionsInRange(TableName tableName, byte[] startKey, + byte[] endKey) throws IOException { + List regionsInRange = new ArrayList(); + byte[] currentKey = startKey; + final boolean endKeyIsEndOfTable = Bytes.equals(endKey, HConstants.EMPTY_END_ROW); + try (RegionLocator r = TEST_UTIL.getConnection().getRegionLocator(tableName);) { + do { + HRegionLocation regionLocation = r.getRegionLocation(currentKey); + regionsInRange.add(regionLocation); + currentKey = regionLocation.getRegionInfo().getEndKey(); + } while (!Bytes.equals(currentKey, HConstants.EMPTY_END_ROW) + && (endKeyIsEndOfTable || Bytes.compareTo(currentKey, endKey) < 0)); + return regionsInRange; + } + } + @Test public void testJira6912() throws Exception { TableName TABLE = TableName.valueOf("testJira6912"); @@ -6080,9 +6128,10 @@ public class TestFromClientSide { Bytes.add(Bytes.toBytes("007"), Bytes.multiple(maxByteArray, 4)), Bytes.toBytes("008"), Bytes.multiple(maxByteArray, 2) }; HTable table = TEST_UTIL.createTable(TABLE, FAMILY, splitRows); - TEST_UTIL.waitUntilAllRegionsAssigned(table.getName()); - - assertEquals(splitRows.length + 1, table.getRegionLocations().size()); + TEST_UTIL.waitUntilAllRegionsAssigned(TABLE); + try (RegionLocator r = TEST_UTIL.getConnection().getRegionLocator(TABLE)) { + assertEquals(splitRows.length + 1, r.getAllRegionLocations().size()); + } // Insert one row each region int insertNum = splitRows.length; for (int i = 0; i < insertNum; i++) { @@ -6133,8 +6182,9 @@ public class TestFromClientSide { Bytes.toBytes("006"), Bytes.toBytes("008"), Bytes.toBytes("010")}; HTable table = TEST_UTIL.createTable(TABLE, FAMILY, splitRows); TEST_UTIL.waitUntilAllRegionsAssigned(table.getName()); - - assertEquals(splitRows.length + 1, table.getRegionLocations().size()); + try (RegionLocator r = TEST_UTIL.getConnection().getRegionLocator(TABLE)) { + assertEquals(splitRows.length + 1, r.getAllRegionLocations().size()); + } for (byte[] splitRow : splitRows) { Put put = new Put(splitRow); put.add(FAMILY, QUALIFIER, VALUE); diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestFromClientSide3.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestFromClientSide3.java index 1e7fbc7..608a62a 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestFromClientSide3.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestFromClientSide3.java @@ -20,9 +20,9 @@ package org.apache.hadoop.hbase.client; import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertFalse; import static org.junit.Assert.assertNull; import static org.junit.Assert.assertTrue; -import static org.junit.Assert.assertFalse; import static org.junit.Assert.fail; import java.util.ArrayList; @@ -32,11 +32,12 @@ import java.util.Random; import org.apache.commons.logging.Log; import org.apache.commons.logging.LogFactory; -import org.apache.hadoop.hbase.TableName; +import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.hbase.HBaseTestingUtility; import org.apache.hadoop.hbase.HColumnDescriptor; import org.apache.hadoop.hbase.HRegionLocation; import org.apache.hadoop.hbase.HTableDescriptor; +import org.apache.hadoop.hbase.TableName; import org.apache.hadoop.hbase.protobuf.ProtobufUtil; import org.apache.hadoop.hbase.protobuf.generated.AdminProtos; import org.apache.hadoop.hbase.testclassification.ClientTests; @@ -118,24 +119,26 @@ public class TestFromClientSide3 { byte[] row, byte[] family, int nFlushes, int nPuts) throws Exception { - // connection needed for poll-wait - HRegionLocation loc = table.getRegionLocation(row, true); - AdminProtos.AdminService.BlockingInterface server = - admin.getConnection().getAdmin(loc.getServerName()); - byte[] regName = loc.getRegionInfo().getRegionName(); - - for (int i = 0; i < nFlushes; i++) { - randomCFPuts(table, row, family, nPuts); - List sf = ProtobufUtil.getStoreFiles(server, regName, FAMILY); - int sfCount = sf.size(); - - // TODO: replace this api with a synchronous flush after HBASE-2949 - admin.flush(table.getName()); - - // synchronously poll wait for a new storefile to appear (flush happened) - while (ProtobufUtil.getStoreFiles( - server, regName, FAMILY).size() == sfCount) { - Thread.sleep(40); + Connection c = TEST_UTIL.getConnection(); + try (RegionLocator r = c.getRegionLocator(table.getName())) { + HRegionLocation loc = r.getRegionLocation(row, true); + // connection needed for poll-wait + AdminProtos.AdminService.BlockingInterface server = + admin.getConnection().getAdmin(loc.getServerName()); + byte[] regName = loc.getRegionInfo().getRegionName(); + + for (int i = 0; i < nFlushes; i++) { + randomCFPuts(table, row, family, nPuts); + List sf = ProtobufUtil.getStoreFiles(server, regName, FAMILY); + int sfCount = sf.size(); + + // TODO: replace this api with a synchronous flush after HBASE-2949 + admin.flush(table.getName()); + + // synchronously poll wait for a new storefile to appear (flush happened) + while (ProtobufUtil.getStoreFiles(server, regName, FAMILY).size() == sfCount) { + Thread.sleep(40); + } } } } @@ -152,7 +155,8 @@ public class TestFromClientSide3 { * and modify table. The CF schema should override the Table schema and now * cause a minor compaction. */ - TEST_UTIL.getConfiguration().setInt("hbase.hstore.compaction.min", 3); + Configuration conf = TEST_UTIL.getConfiguration(); + conf.setInt("hbase.hstore.compaction.min", 3); String tableName = "testAdvancedConfigOverride"; TableName TABLE = TableName.valueOf(tableName); @@ -164,110 +168,107 @@ public class TestFromClientSide3 { byte[] row = Bytes.toBytes(random.nextInt()); performMultiplePutAndFlush(admin, hTable, row, FAMILY, 3, 100); - // Verify we have multiple store files. - HRegionLocation loc = hTable.getRegionLocation(row, true); - byte[] regionName = loc.getRegionInfo().getRegionName(); - AdminProtos.AdminService.BlockingInterface server = - connection.getAdmin(loc.getServerName()); - assertTrue(ProtobufUtil.getStoreFiles( - server, regionName, FAMILY).size() > 1); - - // Issue a compaction request - admin.compact(TABLE.getName()); - - // poll wait for the compactions to happen - for (int i = 0; i < 10 * 1000 / 40; ++i) { - // The number of store files after compaction should be lesser. - loc = hTable.getRegionLocation(row, true); - if (!loc.getRegionInfo().isOffline()) { - regionName = loc.getRegionInfo().getRegionName(); - server = connection.getAdmin(loc.getServerName()); - if (ProtobufUtil.getStoreFiles( - server, regionName, FAMILY).size() <= 1) { - break; + Connection c = TEST_UTIL.getConnection(); + try (RegionLocator r = c.getRegionLocator(TABLE)) { + // Verify we have multiple store files. + HRegionLocation loc = r.getRegionLocation(row, true); + byte[] regionName = loc.getRegionInfo().getRegionName(); + AdminProtos.AdminService.BlockingInterface server = connection.getAdmin(loc.getServerName()); + assertTrue(ProtobufUtil.getStoreFiles(server, regionName, FAMILY).size() > 1); + + // Issue a compaction request + admin.compact(TABLE.getName()); + + // poll wait for the compactions to happen + for (int i = 0; i < 10 * 1000 / 40; ++i) { + // The number of store files after compaction should be lesser. + loc = r.getRegionLocation(row, true); + if (!loc.getRegionInfo().isOffline()) { + regionName = loc.getRegionInfo().getRegionName(); + server = connection.getAdmin(loc.getServerName()); + if (ProtobufUtil.getStoreFiles(server, regionName, FAMILY).size() <= 1) { + break; + } } + Thread.sleep(40); } - Thread.sleep(40); - } - // verify the compactions took place and that we didn't just time out - assertTrue(ProtobufUtil.getStoreFiles( - server, regionName, FAMILY).size() <= 1); - - // change the compaction.min config option for this table to 5 - LOG.info("hbase.hstore.compaction.min should now be 5"); - HTableDescriptor htd = new HTableDescriptor(hTable.getTableDescriptor()); - htd.setValue("hbase.hstore.compaction.min", String.valueOf(5)); - admin.modifyTable(TABLE, htd); - Pair st; - while (null != (st = admin.getAlterStatus(TABLE)) && st.getFirst() > 0) { - LOG.debug(st.getFirst() + " regions left to update"); - Thread.sleep(40); - } - LOG.info("alter status finished"); - - // Create 3 more store files. - performMultiplePutAndFlush(admin, hTable, row, FAMILY, 3, 10); - - // Issue a compaction request - admin.compact(TABLE.getName()); - - // This time, the compaction request should not happen - Thread.sleep(10 * 1000); - loc = hTable.getRegionLocation(row, true); - regionName = loc.getRegionInfo().getRegionName(); - server = connection.getAdmin(loc.getServerName()); - int sfCount = ProtobufUtil.getStoreFiles( - server, regionName, FAMILY).size(); - assertTrue(sfCount > 1); - - // change an individual CF's config option to 2 & online schema update - LOG.info("hbase.hstore.compaction.min should now be 2"); - HColumnDescriptor hcd = new HColumnDescriptor(htd.getFamily(FAMILY)); - hcd.setValue("hbase.hstore.compaction.min", String.valueOf(2)); - htd.modifyFamily(hcd); - admin.modifyTable(TABLE, htd); - while (null != (st = admin.getAlterStatus(TABLE)) && st.getFirst() > 0) { - LOG.debug(st.getFirst() + " regions left to update"); - Thread.sleep(40); - } - LOG.info("alter status finished"); + // verify the compactions took place and that we didn't just time out + assertTrue(ProtobufUtil.getStoreFiles(server, regionName, FAMILY).size() <= 1); + + // change the compaction.min config option for this table to 5 + LOG.info("hbase.hstore.compaction.min should now be 5"); + HTableDescriptor htd = new HTableDescriptor(hTable.getTableDescriptor()); + htd.setValue("hbase.hstore.compaction.min", String.valueOf(5)); + admin.modifyTable(TABLE, htd); + Pair st; + while (null != (st = admin.getAlterStatus(TABLE)) && st.getFirst() > 0) { + LOG.debug(st.getFirst() + " regions left to update"); + Thread.sleep(40); + } + LOG.info("alter status finished"); - // Issue a compaction request - admin.compact(TABLE.getName()); + // Create 3 more store files. + performMultiplePutAndFlush(admin, hTable, row, FAMILY, 3, 10); - // poll wait for the compactions to happen - for (int i = 0; i < 10 * 1000 / 40; ++i) { - loc = hTable.getRegionLocation(row, true); + // Issue a compaction request + admin.compact(TABLE.getName()); + + // This time, the compaction request should not happen + Thread.sleep(10 * 1000); + loc = r.getRegionLocation(row, true); regionName = loc.getRegionInfo().getRegionName(); - try { - server = connection.getAdmin(loc.getServerName()); - if (ProtobufUtil.getStoreFiles( - server, regionName, FAMILY).size() < sfCount) { - break; + server = connection.getAdmin(loc.getServerName()); + int sfCount = ProtobufUtil.getStoreFiles(server, regionName, FAMILY).size(); + assertTrue(sfCount > 1); + + // change an individual CF's config option to 2 & online schema update + LOG.info("hbase.hstore.compaction.min should now be 2"); + HColumnDescriptor hcd = new HColumnDescriptor(htd.getFamily(FAMILY)); + hcd.setValue("hbase.hstore.compaction.min", String.valueOf(2)); + htd.modifyFamily(hcd); + admin.modifyTable(TABLE, htd); + while (null != (st = admin.getAlterStatus(TABLE)) && st.getFirst() > 0) { + LOG.debug(st.getFirst() + " regions left to update"); + Thread.sleep(40); + } + LOG.info("alter status finished"); + + // Issue a compaction request + admin.compact(TABLE.getName()); + + // poll wait for the compactions to happen + for (int i = 0; i < 10 * 1000 / 40; ++i) { + loc = r.getRegionLocation(row, true); + regionName = loc.getRegionInfo().getRegionName(); + try { + server = connection.getAdmin(loc.getServerName()); + if (ProtobufUtil.getStoreFiles(server, regionName, FAMILY).size() < sfCount) { + break; + } + } catch (Exception e) { + LOG.debug("Waiting for region to come online: " + regionName); } - } catch (Exception e) { - LOG.debug("Waiting for region to come online: " + regionName); + Thread.sleep(40); } - Thread.sleep(40); - } - // verify the compaction took place and that we didn't just time out - assertTrue(ProtobufUtil.getStoreFiles( - server, regionName, FAMILY).size() < sfCount); - - // Finally, ensure that we can remove a custom config value after we made it - LOG.info("Removing CF config value"); - LOG.info("hbase.hstore.compaction.min should now be 5"); - hcd = new HColumnDescriptor(htd.getFamily(FAMILY)); - hcd.setValue("hbase.hstore.compaction.min", null); - htd.modifyFamily(hcd); - admin.modifyTable(TABLE, htd); - while (null != (st = admin.getAlterStatus(TABLE)) && st.getFirst() > 0) { - LOG.debug(st.getFirst() + " regions left to update"); - Thread.sleep(40); + + // verify the compaction took place and that we didn't just time out + assertTrue(ProtobufUtil.getStoreFiles(server, regionName, FAMILY).size() < sfCount); + + // Finally, ensure that we can remove a custom config value after we made it + LOG.info("Removing CF config value"); + LOG.info("hbase.hstore.compaction.min should now be 5"); + hcd = new HColumnDescriptor(htd.getFamily(FAMILY)); + hcd.setValue("hbase.hstore.compaction.min", null); + htd.modifyFamily(hcd); + admin.modifyTable(TABLE, htd); + while (null != (st = admin.getAlterStatus(TABLE)) && st.getFirst() > 0) { + LOG.debug(st.getFirst() + " regions left to update"); + Thread.sleep(40); + } + LOG.info("alter status finished"); + assertNull(hTable.getTableDescriptor().getFamily(FAMILY) + .getValue("hbase.hstore.compaction.min")); } - LOG.info("alter status finished"); - assertNull(hTable.getTableDescriptor().getFamily(FAMILY).getValue( - "hbase.hstore.compaction.min")); } @Test @@ -330,7 +331,7 @@ public class TestFromClientSide3 { gets.add(null); gets.add(new Get(ANOTHERROW)); - Boolean[] results = table.exists(gets); + boolean[] results = table.existsAll(gets); assertEquals(results[0], true); assertEquals(results[1], false); assertEquals(results[2], false); @@ -427,7 +428,7 @@ public class TestFromClientSide3 { gets.add(new Get(Bytes.add(ANOTHERROW, new byte[] { 0x00 }))); LOG.info("Calling exists"); - Boolean[] results = table.exists(gets); + boolean[] results = table.existsAll(gets); assertEquals(results[0], false); assertEquals(results[1], false); assertEquals(results[2], true); @@ -441,7 +442,7 @@ public class TestFromClientSide3 { gets = new ArrayList(); gets.add(new Get(new byte[] { 0x00 })); gets.add(new Get(new byte[] { 0x00, 0x00 })); - results = table.exists(gets); + results = table.existsAll(gets); assertEquals(results[0], true); assertEquals(results[1], false); @@ -454,7 +455,7 @@ public class TestFromClientSide3 { gets.add(new Get(new byte[] { (byte) 0xff })); gets.add(new Get(new byte[] { (byte) 0xff, (byte) 0xff })); gets.add(new Get(new byte[] { (byte) 0xff, (byte) 0xff, (byte) 0xff })); - results = table.exists(gets); + results = table.existsAll(gets); assertEquals(results[0], false); assertEquals(results[1], true); assertEquals(results[2], false); 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 f5807c2..760b592 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 @@ -81,7 +81,10 @@ public class TestFromClientSideNoCodec { } // Check getRowOrBefore byte [] f = fs[0]; - r = ht.getRowOrBefore(row, f); + Get get = new Get(row); + get.setClosestRowBefore(true); + get.addFamily(f); + r = ht.get(get); assertTrue(r.toString(), r.containsColumn(f, f)); // Check scan. ResultScanner scanner = ht.getScanner(new Scan()); 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 9ed5be6..1b0907e 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 @@ -28,6 +28,7 @@ import java.io.IOException; import java.lang.reflect.Field; import java.lang.reflect.Modifier; import java.net.SocketTimeoutException; +import java.util.ArrayList; import java.util.List; import java.util.Random; import java.util.concurrent.ExecutorService; @@ -221,18 +222,19 @@ public class TestHCM { getRegionStates().isRegionsInTransition()){ Thread.sleep(1); } - final ConnectionImplementation hci = (ConnectionImplementation)t.getConnection(); - while (t.getRegionLocation(rk).getPort() != sn.getPort()){ - TEST_UTIL.getHBaseAdmin().move(t.getRegionLocation(rk).getRegionInfo(). - getEncodedNameAsBytes(), Bytes.toBytes(sn.toString())); - while(TEST_UTIL.getHBaseCluster().getMaster().getAssignmentManager(). - getRegionStates().isRegionsInTransition()){ + final ConnectionImplementation hci = (ConnectionImplementation) t.getConnection(); + HRegionLocation regionLocation = hci.getRegionLocation(tn, rk, false); + while (regionLocation.getPort() != sn.getPort()) { + TEST_UTIL.getHBaseAdmin().move(regionLocation.getRegionInfo().getEncodedNameAsBytes(), + Bytes.toBytes(sn.toString())); + while (TEST_UTIL.getHBaseCluster().getMaster().getAssignmentManager().getRegionStates() + .isRegionsInTransition()) { Thread.sleep(1); } hci.clearRegionCache(tn); } Assert.assertNotNull(hci.clusterStatusListener); - TEST_UTIL.assertRegionOnServer(t.getRegionLocation(rk).getRegionInfo(), sn, 20000); + TEST_UTIL.assertRegionOnServer(regionLocation.getRegionInfo(), sn, 20000); Put p1 = new Put(rk); p1.add(cf, "qual".getBytes(), "val".getBytes()); @@ -364,17 +366,19 @@ public class TestHCM { } }); - ServerName sn = table.getRegionLocation(ROW).getServerName(); ConnectionImplementation conn = (ConnectionImplementation) table.getConnection(); - RpcClient rpcClient = conn.getRpcClient(); + try (RegionLocator r = conn.getRegionLocator(tableName);) { + ServerName sn = r.getRegionLocation(ROW).getServerName(); - LOG.info("Going to cancel connections. connection=" + conn.toString() + ", sn=" + sn); - for (int i = 0; i < 5000; i++) { - rpcClient.cancelConnections(sn); - Thread.sleep(5); - } + RpcClient rpcClient = conn.getRpcClient(); + LOG.info("Going to cancel connections. connection=" + conn.toString() + ", sn=" + sn); + for (int i = 0; i < 5000; i++) { + rpcClient.cancelConnections(sn); + Thread.sleep(5); + } + } step.compareAndSet(1, 2); // The test may fail here if the thread doing the gets is stuck. The way to find // out what's happening is to look for the thread named 'testConnectionCloseThread' @@ -476,7 +480,7 @@ public class TestHCM { table.put(p); final ConnectionImplementation hci = (ConnectionImplementation)table.getConnection(); - final HRegionLocation loc = table.getRegionLocation(FAM_NAM); + final HRegionLocation loc = hci.getRegionLocation(tableName, FAM_NAM, false); Get get = new Get(FAM_NAM); Assert.assertNotNull(table.get(get)); @@ -970,8 +974,10 @@ public class TestHCM { Put put4 = new Put(otherRow); put4.add(FAM_NAM, otherRow, otherRow); - // do multi - table.batch(Lists.newArrayList(put4, put3)); // first should be a valid row, + // do multi + ArrayList actions = Lists.newArrayList(put4, put3); + Object[] result = new Object[actions.size()]; + table.batch(actions, result); // first should be a valid row, // second we get RegionMovedException. setNumTries(conn, prevNumRetriesVal); diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestHTableMultiplexer.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestHTableMultiplexer.java index 43ba242..281fb86 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestHTableMultiplexer.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestHTableMultiplexer.java @@ -89,8 +89,8 @@ public class TestHTableMultiplexer { final int VERSION = 3; List failedPuts; boolean success; - - HTableMultiplexer multiplexer = new HTableMultiplexer(TEST_UTIL.getConfiguration(), + + HTableMultiplexer multiplexer = new HTableMultiplexer(TEST_UTIL.getConfiguration(), PER_REGIONSERVER_QUEUE_SIZE); HTable htable1 = @@ -101,55 +101,57 @@ public class TestHTableMultiplexer { Bytes.toBytes("zzzzz"), NUM_REGIONS); TEST_UTIL.waitUntilAllRegionsAssigned(TABLE_1); TEST_UTIL.waitUntilAllRegionsAssigned(TABLE_2); - - byte[][] startRows = htable1.getStartKeys(); - byte[][] endRows = htable1.getEndKeys(); - - // SinglePut case - for (int i = 0; i < NUM_REGIONS; i++) { - byte [] row = startRows[i]; - if (row == null || row.length <= 0) continue; - Put put = new Put(row).add(FAMILY, QUALIFIER, VALUE1); - success = multiplexer.put(TABLE_1, put); - assertTrue("multiplexer.put returns", success); - - put = new Put(row).add(FAMILY, QUALIFIER, VALUE1); - success = multiplexer.put(TABLE_2, put); - assertTrue("multiplexer.put failed", success); - - LOG.info("Put for " + Bytes.toStringBinary(startRows[i]) + " @ iteration " + (i + 1)); + Connection c = TEST_UTIL.getConnection(); + try (RegionLocator rl = c.getRegionLocator(TABLE_1)) { + byte[][] startRows = rl.getStartKeys(); + byte[][] endRows = rl.getEndKeys(); + + // SinglePut case + for (int i = 0; i < NUM_REGIONS; i++) { + byte[] row = startRows[i]; + if (row == null || row.length <= 0) continue; + Put put = new Put(row).add(FAMILY, QUALIFIER, VALUE1); + success = multiplexer.put(TABLE_1, put); + assertTrue("multiplexer.put returns", success); + + put = new Put(row).add(FAMILY, QUALIFIER, VALUE1); + success = multiplexer.put(TABLE_2, put); + assertTrue("multiplexer.put failed", success); + + LOG.info("Put for " + Bytes.toStringBinary(startRows[i]) + " @ iteration " + (i + 1)); + + // verify that the Get returns the correct result + checkExistence(htable1, startRows[i], FAMILY, QUALIFIER); + checkExistence(htable2, startRows[i], FAMILY, QUALIFIER); + } + + // MultiPut case + List multiput = new ArrayList(); + for (int i = 0; i < NUM_REGIONS; i++) { + byte[] row = endRows[i]; + if (row == null || row.length <= 0) continue; + Put put = new Put(row); + put.add(FAMILY, QUALIFIER, VALUE2); + multiput.add(put); + } + failedPuts = multiplexer.put(TABLE_1, multiput); + assertTrue(failedPuts == null); // verify that the Get returns the correct result - checkExistence(htable1, startRows[i], FAMILY, QUALIFIER); - checkExistence(htable2, startRows[i], FAMILY, QUALIFIER); - } - - // MultiPut case - List multiput = new ArrayList(); - for (int i = 0; i < NUM_REGIONS; i++) { - byte [] row = endRows[i]; - if (row == null || row.length <= 0) continue; - Put put = new Put(row); - put.add(FAMILY, QUALIFIER, VALUE2); - multiput.add(put); - } - failedPuts = multiplexer.put(TABLE_1, multiput); - assertTrue(failedPuts == null); - - // verify that the Get returns the correct result - for (int i = 0; i < NUM_REGIONS; i++) { - byte [] row = endRows[i]; - if (row == null || row.length <= 0) continue; - Get get = new Get(row); - get.addColumn(FAMILY, QUALIFIER); - Result r; - int nbTry = 0; - do { - assertTrue(nbTry++ < 50); - Thread.sleep(100); - r = htable1.get(get); - } while (r == null || r.getValue(FAMILY, QUALIFIER) == null || - Bytes.compareTo(VALUE2, r.getValue(FAMILY, QUALIFIER)) != 0); + for (int i = 0; i < NUM_REGIONS; i++) { + byte[] row = endRows[i]; + if (row == null || row.length <= 0) continue; + Get get = new Get(row); + get.addColumn(FAMILY, QUALIFIER); + Result r; + int nbTry = 0; + do { + assertTrue(nbTry++ < 50); + Thread.sleep(100); + r = htable1.get(get); + } while (r == null || r.getValue(FAMILY, QUALIFIER) == null + || Bytes.compareTo(VALUE2, r.getValue(FAMILY, QUALIFIER)) != 0); + } } } } 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 4091e58..20f2ee5 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 @@ -19,8 +19,11 @@ */ package org.apache.hadoop.hbase.client; +import static org.junit.Assert.assertTrue; + import org.apache.commons.logging.Log; import org.apache.commons.logging.LogFactory; +import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.hbase.HBaseTestingUtility; import org.apache.hadoop.hbase.HRegionLocation; import org.apache.hadoop.hbase.MiniHBaseCluster; @@ -34,8 +37,6 @@ import org.junit.BeforeClass; import org.junit.Test; import org.junit.experimental.categories.Category; -import static org.junit.Assert.assertTrue; - @Category({ LargeTests.class, ClientTests.class }) public class TestHTableMultiplexerFlushCache { private static final Log LOG = LogFactory.getLog(TestHTableMultiplexerFlushCache.class); @@ -89,28 +90,31 @@ public class TestHTableMultiplexerFlushCache { HTable htable = TEST_UTIL.createTable(TABLE, new byte[][] { FAMILY }, 3, Bytes.toBytes("aaaaa"), Bytes.toBytes("zzzzz"), NUM_REGIONS); - HTableMultiplexer multiplexer = new HTableMultiplexer(TEST_UTIL.getConfiguration(), + Configuration conf = TEST_UTIL.getConfiguration(); + HTableMultiplexer multiplexer = new HTableMultiplexer(conf, PER_REGIONSERVER_QUEUE_SIZE); - - byte[][] startRows = htable.getStartKeys(); - byte[] row = startRows[1]; - assertTrue("2nd region should not start with empty row", row != null && row.length > 0); + Connection c = TEST_UTIL.getConnection(); + try (RegionLocator r = c.getRegionLocator(TABLE)) { + byte[][] startRows = r.getStartKeys(); + byte[] row = startRows[1]; + assertTrue("2nd region should not start with empty row", row != null && row.length > 0); + + Put put = new Put(row).add(FAMILY, QUALIFIER1, VALUE1); + assertTrue("multiplexer.put returns", multiplexer.put(TABLE, put)); - Put put = new Put(row).add(FAMILY, QUALIFIER1, VALUE1); - assertTrue("multiplexer.put returns", multiplexer.put(TABLE, put)); - - checkExistence(htable, row, FAMILY, QUALIFIER1, VALUE1); + checkExistence(htable, row, FAMILY, QUALIFIER1, VALUE1); - // Now let's shutdown the regionserver and let regions moved to other servers. - HRegionLocation loc = htable.getRegionLocation(row); - MiniHBaseCluster hbaseCluster = TEST_UTIL.getHBaseCluster(); - hbaseCluster.stopRegionServer(loc.getServerName()); - TEST_UTIL.waitUntilAllRegionsAssigned(TABLE); + // Now let's shutdown the regionserver and let regions moved to other servers. + HRegionLocation loc = r.getRegionLocation(row); + MiniHBaseCluster hbaseCluster = TEST_UTIL.getHBaseCluster(); + hbaseCluster.stopRegionServer(loc.getServerName()); + TEST_UTIL.waitUntilAllRegionsAssigned(TABLE); - // put with multiplexer. - put = new Put(row).add(FAMILY, QUALIFIER2, VALUE2); - assertTrue("multiplexer.put returns", multiplexer.put(TABLE, put)); + // put with multiplexer. + put = new Put(row).add(FAMILY, QUALIFIER2, VALUE2); + assertTrue("multiplexer.put returns", multiplexer.put(TABLE, put)); - checkExistence(htable, row, FAMILY, QUALIFIER2, VALUE2); + checkExistence(htable, row, FAMILY, QUALIFIER2, VALUE2); + } } } diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestMultiParallel.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestMultiParallel.java index 2958834..1ddd5e2 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestMultiParallel.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestMultiParallel.java @@ -150,7 +150,8 @@ public class TestMultiParallel { try { try (Table t = connection.getTable(TEST_TABLE, executor)) { List puts = constructPutRequests(); // creates a Put for every region - t.batch(puts); + Object[] result = new Object[puts.size()]; + t.batch(puts, result); HashSet regionservers = new HashSet(); try (RegionLocator locator = connection.getRegionLocator(TEST_TABLE)) { for (Row r : puts) { @@ -173,7 +174,8 @@ public class TestMultiParallel { // load test data List puts = constructPutRequests(); - table.batch(puts); + Object[] results = new Object[puts.size()]; + table.batch(puts, results); // create a list of gets and run it List gets = new ArrayList(); @@ -326,7 +328,8 @@ public class TestMultiParallel { // put multiple rows using a batch List puts = constructPutRequests(); - Object[] results = table.batch(puts); + Object[] results = new Object[puts.size()]; + table.batch(puts, results); validateSizeAndEmpty(results, KEYS.length); if (true) { @@ -337,7 +340,8 @@ public class TestMultiParallel { liveRS.getRegionServer().abort("Aborting for tests", new Exception("testBatchWithPut")); puts = constructPutRequests(); try { - results = table.batch(puts); + results = new Object[puts.size()]; + table.batch(puts, results); } catch (RetriesExhaustedWithDetailsException ree) { LOG.info(ree.getExhaustiveDescription()); table.close(); @@ -357,7 +361,8 @@ public class TestMultiParallel { // Load some data List puts = constructPutRequests(); - Object[] results = table.batch(puts); + Object[] results = new Object[puts.size()]; + table.batch(puts, results); validateSizeAndEmpty(results, KEYS.length); // Deletes @@ -367,7 +372,8 @@ public class TestMultiParallel { delete.addFamily(BYTES_FAMILY); deletes.add(delete); } - results = table.batch(deletes); + results= new Object[deletes.size()]; + table.batch(deletes, results); validateSizeAndEmpty(results, KEYS.length); // Get to make sure ... @@ -386,7 +392,8 @@ public class TestMultiParallel { // Load some data List puts = constructPutRequests(); - Object[] results = table.batch(puts); + Object[] results = new Object[puts.size()]; + table.batch(puts, results); validateSizeAndEmpty(results, KEYS.length); // Deletes @@ -420,7 +427,8 @@ public class TestMultiParallel { put.add(BYTES_FAMILY, qual, VALUE); puts.add(put); } - Object[] results = table.batch(puts); + Object[] results = new Object[puts.size()]; + table.batch(puts, results); // validate validateSizeAndEmpty(results, 100); @@ -434,7 +442,8 @@ public class TestMultiParallel { gets.add(get); } - Object[] multiRes = table.batch(gets); + Object[] multiRes = new Object[gets.size()]; + table.batch(gets, multiRes); int idx = 0; for (Object r : multiRes) { @@ -471,7 +480,8 @@ public class TestMultiParallel { actions.add(inc); actions.add(a); - Object[] multiRes = table.batch(actions); + Object[] multiRes = new Object[actions.size()]; + table.batch(actions, multiRes); validateResult(multiRes[1], QUAL1, Bytes.toBytes("abcdef")); validateResult(multiRes[1], QUAL4, Bytes.toBytes("xyz")); validateResult(multiRes[0], QUAL2, Bytes.toBytes(2L)); @@ -577,7 +587,9 @@ public class TestMultiParallel { Table table = UTIL.getConnection().getTable(TEST_TABLE); // Load some data to start - Object[] results = table.batch(constructPutRequests()); + List puts = constructPutRequests(); + Object[] results = new Object[puts.size()]; + table.batch(puts, results); validateSizeAndEmpty(results, KEYS.length); // Batch: get, get, put(new col), delete, get, get of put, get of deleted, @@ -621,7 +633,8 @@ public class TestMultiParallel { put.add(BYTES_FAMILY, qual2, val2); actions.add(put); - results = table.batch(actions); + results = new Object[actions.size()]; + table.batch(actions, results); // Validation diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestReplicasClient.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestReplicasClient.java index bfc1230..56888b0 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestReplicasClient.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestReplicasClient.java @@ -45,6 +45,7 @@ import org.apache.hadoop.hbase.HTableDescriptor; import org.apache.hadoop.hbase.KeyValue; import org.apache.hadoop.hbase.NotServingRegionException; import org.apache.hadoop.hbase.RegionLocations; +import org.apache.hadoop.hbase.TableName; import org.apache.hadoop.hbase.TableNotFoundException; import org.apache.hadoop.hbase.client.AsyncProcess.AsyncRequestFuture; import org.apache.hadoop.hbase.client.AsyncProcess.AsyncRequestFutureImpl; @@ -168,18 +169,24 @@ public class TestReplicasClient { @BeforeClass public static void beforeClass() throws Exception { // enable store file refreshing - HTU.getConfiguration().setInt( + Configuration conf = HTU.getConfiguration(); + conf.setInt( StorefileRefresherChore.REGIONSERVER_STOREFILE_REFRESH_PERIOD, REFRESH_PERIOD); - HTU.getConfiguration().setBoolean("hbase.client.log.scanner.activity", true); - ConnectionUtils.setupMasterlessConnection(HTU.getConfiguration()); + conf.setBoolean("hbase.client.log.scanner.activity", true); + ConnectionUtils.setupMasterlessConnection(conf); HTU.startMiniCluster(NB_SERVERS); // Create table then get the single region for our new table. - HTableDescriptor hdt = HTU.createTableDescriptor(TestReplicasClient.class.getSimpleName()); + String name = TestReplicasClient.class.getSimpleName(); + HTableDescriptor hdt = HTU.createTableDescriptor(name); hdt.addCoprocessor(SlowMeCopro.class.getName()); - table = HTU.createTable(hdt, new byte[][]{f}, HTU.getConfiguration()); + table = HTU.createTable(hdt, new byte[][]{f}, conf); - hriPrimary = table.getRegionLocation(row, false).getRegionInfo(); + Configuration c = new Configuration(conf); + try (Connection conn = ConnectionFactory.createConnection(conf); + RegionLocator r = conn.getRegionLocator(TableName.valueOf(name))) { + hriPrimary = r.getRegionLocation(row, false).getRegionInfo(); + } // mock a secondary region info to open hriSecondary = new HRegionInfo(hriPrimary.getTable(), hriPrimary.getStartKey(), @@ -188,7 +195,6 @@ public class TestReplicasClient { // No master LOG.info("Master is going to be stopped"); TestRegionServerNoMaster.stopMasterAndAssignMeta(HTU); - Configuration c = new Configuration(HTU.getConfiguration()); c.setInt(HConstants.HBASE_CLIENT_RETRIES_NUMBER, 1); LOG.info("Master has stopped"); } diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestScannersFromClientSide.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestScannersFromClientSide.java index 5afc226..d5c3fd5 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestScannersFromClientSide.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestScannersFromClientSide.java @@ -285,7 +285,7 @@ public class TestScannersFromClientSide { private void verifyExpectedCounts(Table table, Scan scan, int expectedRowCount, int expectedCellCount) throws Exception { ResultScanner scanner = table.getScanner(scan); - + int rowCount = 0; int cellCount = 0; Result r = null; @@ -599,39 +599,41 @@ public class TestScannersFromClientSide { scan = new Scan(ROW); scanner = ht.getScanner(scan); - HRegionLocation loc = ht.getRegionLocation(ROW); - HRegionInfo hri = loc.getRegionInfo(); - MiniHBaseCluster cluster = TEST_UTIL.getMiniHBaseCluster(); - byte[] regionName = hri.getRegionName(); - int i = cluster.getServerWith(regionName); - HRegionServer rs = cluster.getRegionServer(i); - ProtobufUtil.closeRegion( - rs.getRSRpcServices(), rs.getServerName(), regionName); - long startTime = EnvironmentEdgeManager.currentTime(); - long timeOut = 300000; - while (true) { - if (rs.getOnlineRegion(regionName) == null) { - break; + Connection c = TEST_UTIL.getConnection(); + try (RegionLocator r = c.getRegionLocator(TABLE)) { + HRegionLocation loc = r.getRegionLocation(ROW); + HRegionInfo hri = loc.getRegionInfo(); + MiniHBaseCluster cluster = TEST_UTIL.getMiniHBaseCluster(); + byte[] regionName = hri.getRegionName(); + int i = cluster.getServerWith(regionName); + HRegionServer rs = cluster.getRegionServer(i); + ProtobufUtil.closeRegion(rs.getRSRpcServices(), rs.getServerName(), regionName); + long startTime = EnvironmentEdgeManager.currentTime(); + long timeOut = 300000; + while (true) { + if (rs.getOnlineRegion(regionName) == null) { + break; + } + assertTrue("Timed out in closing the testing region", + EnvironmentEdgeManager.currentTime() < startTime + timeOut); + Thread.sleep(500); } - assertTrue("Timed out in closing the testing region", - EnvironmentEdgeManager.currentTime() < startTime + timeOut); - Thread.sleep(500); - } - // Now open the region again. - HMaster master = cluster.getMaster(); - RegionStates states = master.getAssignmentManager().getRegionStates(); - states.regionOffline(hri); - states.updateRegionState(hri, State.OPENING); - ProtobufUtil.openRegion(rs.getRSRpcServices(), rs.getServerName(), hri); - startTime = EnvironmentEdgeManager.currentTime(); - while (true) { - if (rs.getOnlineRegion(regionName) != null) { - break; + // Now open the region again. + HMaster master = cluster.getMaster(); + RegionStates states = master.getAssignmentManager().getRegionStates(); + states.regionOffline(hri); + states.updateRegionState(hri, State.OPENING); + ProtobufUtil.openRegion(rs.getRSRpcServices(), rs.getServerName(), hri); + startTime = EnvironmentEdgeManager.currentTime(); + while (true) { + if (rs.getOnlineRegion(regionName) != null) { + break; + } + assertTrue("Timed out in open the testing region", + EnvironmentEdgeManager.currentTime() < startTime + timeOut); + Thread.sleep(500); } - assertTrue("Timed out in open the testing region", - EnvironmentEdgeManager.currentTime() < startTime + timeOut); - Thread.sleep(500); } // c0:0, c1:1 diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestSnapshotCloneIndependence.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestSnapshotCloneIndependence.java index 78e62b0..060227f 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestSnapshotCloneIndependence.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestSnapshotCloneIndependence.java @@ -182,15 +182,17 @@ public class TestSnapshotCloneIndependence { } private static void waitOnSplit(final HTable t, int originalCount) throws Exception { - for (int i = 0; i < 200; i++) { - try { - Thread.sleep(50); - } catch (InterruptedException e) { - // Restore the interrupted status - Thread.currentThread().interrupt(); - } - if (t.getAllRegionLocations().size() > originalCount) { - return; + try (RegionLocator r = UTIL.getConnection().getRegionLocator(t.getName())) { + for (int i = 0; i < 200; i++) { + try { + Thread.sleep(50); + } catch (InterruptedException e) { + // Restore the interrupted status + Thread.currentThread().interrupt(); + } + if (r.getAllRegionLocations().size() > originalCount) { + return; + } } } throw new Exception("Split did not increase the number of regions"); diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/coprocessor/TestCoprocessorEndpoint.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/coprocessor/TestCoprocessorEndpoint.java index 1ca7676..c89bbe4 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/coprocessor/TestCoprocessorEndpoint.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/coprocessor/TestCoprocessorEndpoint.java @@ -26,15 +26,10 @@ import static org.junit.Assert.fail; import java.io.IOException; import java.util.Collections; +import java.util.List; import java.util.Map; -import java.util.NavigableMap; import java.util.TreeMap; -import org.apache.hadoop.hbase.client.Admin; -import org.apache.hadoop.hbase.client.Table; -import org.apache.hadoop.hbase.testclassification.CoprocessorTests; -import org.apache.hadoop.hbase.testclassification.MediumTests; -import org.apache.hadoop.hbase.util.ByteStringer; import org.apache.commons.logging.Log; import org.apache.commons.logging.LogFactory; import org.apache.hadoop.conf.Configuration; @@ -42,11 +37,15 @@ 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.HRegionLocation; import org.apache.hadoop.hbase.HTableDescriptor; -import org.apache.hadoop.hbase.ServerName; import org.apache.hadoop.hbase.TableName; +import org.apache.hadoop.hbase.client.Admin; +import org.apache.hadoop.hbase.client.Connection; import org.apache.hadoop.hbase.client.HTable; import org.apache.hadoop.hbase.client.Put; +import org.apache.hadoop.hbase.client.RegionLocator; +import org.apache.hadoop.hbase.client.Table; import org.apache.hadoop.hbase.client.coprocessor.Batch; import org.apache.hadoop.hbase.coprocessor.protobuf.generated.ColumnAggregationProtos; import org.apache.hadoop.hbase.ipc.BlockingRpcCallback; @@ -54,6 +53,9 @@ import org.apache.hadoop.hbase.ipc.CoprocessorRpcChannel; import org.apache.hadoop.hbase.ipc.ServerRpcController; import org.apache.hadoop.hbase.ipc.protobuf.generated.TestProtos; import org.apache.hadoop.hbase.ipc.protobuf.generated.TestRpcServiceProtos; +import org.apache.hadoop.hbase.testclassification.CoprocessorTests; +import org.apache.hadoop.hbase.testclassification.MediumTests; +import org.apache.hadoop.hbase.util.ByteStringer; import org.apache.hadoop.hbase.util.Bytes; import org.junit.AfterClass; import org.junit.BeforeClass; @@ -173,14 +175,13 @@ public class TestCoprocessorEndpoint { @Test public void testCoprocessorService() throws Throwable { - HTable table = (HTable) util.getConnection().getTable(TEST_TABLE); - NavigableMap regions = table.getRegionLocations(); - + Connection connection = util.getConnection(); + HTable table = (HTable) connection.getTable(TEST_TABLE); final TestProtos.EchoRequestProto request = TestProtos.EchoRequestProto.newBuilder().setMessage("hello").build(); final Map results = Collections.synchronizedMap( new TreeMap(Bytes.BYTES_COMPARATOR)); - try { + try (RegionLocator r = connection.getRegionLocator(TEST_TABLE)){ // scan: for all regions final RpcController controller = new ServerRpcController(); table.coprocessorService(TestRpcServiceProtos.TestProtobufRpcProto.class, @@ -208,10 +209,13 @@ public class TestCoprocessorEndpoint { LOG.info("Got value "+e.getValue()+" for region "+Bytes.toStringBinary(e.getKey())); } assertEquals(3, results.size()); - for (HRegionInfo info : regions.navigableKeySet()) { - LOG.info("Region info is "+info.getRegionNameAsString()); + List regions = r.getAllRegionLocations(); + for (HRegionLocation hrl : regions) { + HRegionInfo info = hrl.getRegionInfo(); + LOG.info("Region info is " + info.getRegionNameAsString()); assertTrue(results.containsKey(info.getRegionName())); } + results.clear(); // scan: for region 2 and region 3 @@ -247,12 +251,12 @@ public class TestCoprocessorEndpoint { @Test public void testCoprocessorServiceNullResponse() throws Throwable { - HTable table = (HTable) util.getConnection().getTable(TEST_TABLE); - NavigableMap regions = table.getRegionLocations(); + Connection connection = util.getConnection(); + HTable table = (HTable) connection.getTable(TEST_TABLE); final TestProtos.EchoRequestProto request = TestProtos.EchoRequestProto.newBuilder().setMessage("hello").build(); - try { + try(RegionLocator r = connection.getRegionLocator(TEST_TABLE)) { // scan: for all regions final RpcController controller = new ServerRpcController(); // test that null results are supported @@ -273,8 +277,10 @@ public class TestCoprocessorEndpoint { LOG.info("Got value "+e.getValue()+" for region "+Bytes.toStringBinary(e.getKey())); } assertEquals(3, results.size()); - for (HRegionInfo info : regions.navigableKeySet()) { - LOG.info("Region info is "+info.getRegionNameAsString()); + List regions = r.getAllRegionLocations(); + for (HRegionLocation hrl : regions) { + HRegionInfo info = hrl.getRegionInfo(); + LOG.info("Region info is " + info.getRegionNameAsString()); assertTrue(results.containsKey(info.getRegionName())); assertNull(results.get(info.getRegionName())); } diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/coprocessor/TestHTableWrapper.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/coprocessor/TestHTableWrapper.java index 8efbfbf..bbfb373 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/coprocessor/TestHTableWrapper.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/coprocessor/TestHTableWrapper.java @@ -22,6 +22,7 @@ package org.apache.hadoop.hbase.coprocessor; import java.io.IOException; import java.util.ArrayList; import java.util.Arrays; +import java.util.List; import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.hbase.Coprocessor; @@ -257,9 +258,11 @@ public class TestHTableWrapper { } private void checkBatch() throws IOException, InterruptedException { - Object[] results1 = hTableInterface.batch(Arrays.asList(new Row[] { - new Increment(ROW_A).addColumn(TEST_FAMILY, qualifierCol1, 2L), - new Increment(ROW_A).addColumn(TEST_FAMILY, qualifierCol1, 2L) })); + List actions = + Arrays.asList(new Row[] { new Increment(ROW_A).addColumn(TEST_FAMILY, qualifierCol1, 2L), + new Increment(ROW_A).addColumn(TEST_FAMILY, qualifierCol1, 2L) }); + Object[] results1 = new Object[actions.size()]; + hTableInterface.batch(actions, results1); assertEquals(2, results1.length); for (Object r2 : results1) { assertTrue(r2 instanceof Result); @@ -267,8 +270,7 @@ public class TestHTableWrapper { checkRowValue(ROW_A, Bytes.toBytes(0L)); Object[] results2 = new Result[2]; hTableInterface.batch( - Arrays.asList(new Row[] { new Increment(ROW_A).addColumn(TEST_FAMILY, qualifierCol1, 2L), - new Increment(ROW_A).addColumn(TEST_FAMILY, qualifierCol1, 2L) }), results2); + actions, results2); for (Object r2 : results2) { assertTrue(r2 instanceof Result); } @@ -277,8 +279,7 @@ public class TestHTableWrapper { // with callbacks: final long[] updateCounter = new long[] { 0L }; Object[] results3 = hTableInterface.batchCallback( - Arrays.asList(new Row[] { new Increment(ROW_A).addColumn(TEST_FAMILY, qualifierCol1, 2L), - new Increment(ROW_A).addColumn(TEST_FAMILY, qualifierCol1, 2L) }), + actions, new Batch.Callback() { @Override public void update(byte[] region, byte[] row, Result result) { @@ -295,8 +296,7 @@ public class TestHTableWrapper { Object[] results4 = new Result[2]; updateCounter[0] = 0L; hTableInterface.batchCallback( - Arrays.asList(new Row[] { new Increment(ROW_A).addColumn(TEST_FAMILY, qualifierCol1, 2L), - new Increment(ROW_A).addColumn(TEST_FAMILY, qualifierCol1, 2L) }), results4, + actions, results4, new Batch.Callback() { @Override public void update(byte[] region, byte[] row, Result result) { diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/coprocessor/TestMasterObserver.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/coprocessor/TestMasterObserver.java index 1f95a53..becfcd4 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/coprocessor/TestMasterObserver.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/coprocessor/TestMasterObserver.java @@ -27,8 +27,6 @@ import static org.junit.Assert.assertTrue; import java.io.IOException; import java.util.Collection; import java.util.List; -import java.util.Map; -import java.util.NavigableMap; import java.util.concurrent.CountDownLatch; import org.apache.commons.logging.Log; @@ -38,6 +36,7 @@ import org.apache.hadoop.hbase.CoprocessorEnvironment; import org.apache.hadoop.hbase.HBaseTestingUtility; import org.apache.hadoop.hbase.HColumnDescriptor; import org.apache.hadoop.hbase.HRegionInfo; +import org.apache.hadoop.hbase.HRegionLocation; import org.apache.hadoop.hbase.HTableDescriptor; import org.apache.hadoop.hbase.MiniHBaseCluster; import org.apache.hadoop.hbase.NamespaceDescriptor; @@ -45,6 +44,7 @@ import org.apache.hadoop.hbase.ServerName; import org.apache.hadoop.hbase.TableName; import org.apache.hadoop.hbase.client.Admin; import org.apache.hadoop.hbase.client.HTable; +import org.apache.hadoop.hbase.client.RegionLocator; import org.apache.hadoop.hbase.master.AssignmentManager; import org.apache.hadoop.hbase.master.HMaster; import org.apache.hadoop.hbase.master.MasterCoprocessorHost; @@ -1612,15 +1612,14 @@ public class TestMasterObserver { cp.enableBypass(false); cp.resetStates(); - HTable table = UTIL.createMultiRegionTable(tableName, TEST_FAMILY); - - try { + UTIL.createMultiRegionTable(tableName, TEST_FAMILY); + try (RegionLocator r = UTIL.getConnection().getRegionLocator(tableName)) { UTIL.waitUntilAllRegionsAssigned(tableName); - NavigableMap regions = table.getRegionLocations(); - Map.Entry firstGoodPair = null; - for (Map.Entry e: regions.entrySet()) { - if (e.getValue() != null) { + List regions = r.getAllRegionLocations(); + HRegionLocation firstGoodPair = null; + for (HRegionLocation e: regions) { + if (e.getServerName() != null) { firstGoodPair = e; break; } @@ -1630,7 +1629,7 @@ public class TestMasterObserver { // Try to force a move Collection servers = master.getClusterStatus().getServers(); String destName = null; - String serverNameForFirstRegion = firstGoodPair.getValue().toString(); + String serverNameForFirstRegion = firstGoodPair.getServerName().toString(); LOG.info("serverNameForFirstRegion=" + serverNameForFirstRegion); ServerName masterServerName = master.getServerName(); boolean found = false; @@ -1647,7 +1646,7 @@ public class TestMasterObserver { assertTrue("Found server", found); LOG.info("Found " + destName); master.getMasterRpcServices().moveRegion(null, RequestConverter.buildMoveRegionRequest( - firstGoodPair.getKey().getEncodedNameAsBytes(),Bytes.toBytes(destName))); + firstGoodPair.getRegionInfo().getEncodedNameAsBytes(),Bytes.toBytes(destName))); assertTrue("Coprocessor should have been called on region move", cp.wasMoveCalled()); diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/coprocessor/TestOpenTableInCoprocessor.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/coprocessor/TestOpenTableInCoprocessor.java index 4c3594b..eb8b2f3 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/coprocessor/TestOpenTableInCoprocessor.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/coprocessor/TestOpenTableInCoprocessor.java @@ -41,6 +41,7 @@ import org.junit.experimental.categories.Category; import java.io.IOException; import java.util.Collections; +import java.util.List; import java.util.concurrent.ExecutorService; import java.util.concurrent.SynchronousQueue; import java.util.concurrent.ThreadPoolExecutor; @@ -104,7 +105,8 @@ public class TestOpenTableInCoprocessor { Put p = new Put(new byte[] { 'a' }); p.add(family, null, new byte[] { 'a' }); try { - table.batch(Collections.singletonList(put)); + List actions = Collections.singletonList(put); + table.batch(actions, new Object[actions.size()]); } catch (InterruptedException e1) { throw new IOException(e1); } 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 d0e561f..1a0f59b 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 @@ -53,6 +53,7 @@ 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; import org.apache.hadoop.hbase.client.Result; import org.apache.hadoop.hbase.client.ResultScanner; import org.apache.hadoop.hbase.client.RowMutations; @@ -596,13 +597,13 @@ public class TestRegionObserverInterface { LOG.info(TestRegionObserverInterface.class.getName() +".testRecovery"); TableName tableName = TableName.valueOf(TEST_TABLE.getNameAsString() + ".testRecovery"); HTable table = util.createTable(tableName, new byte[][] {A, B, C}); - try { + try (RegionLocator r = util.getConnection().getRegionLocator(tableName)) { JVMClusterUtil.RegionServerThread rs1 = cluster.startRegionServer(); ServerName sn2 = rs1.getRegionServer().getServerName(); - String regEN = table.getRegionLocations().firstEntry().getKey().getEncodedName(); + String regEN = r.getAllRegionLocations().get(0).getRegionInfo().getEncodedName(); util.getHBaseAdmin().move(regEN.getBytes(), sn2.getServerName().getBytes()); - while (!sn2.equals(table.getRegionLocations().firstEntry().getValue() )){ + while (!sn2.equals(r.getAllRegionLocations().get(0).getServerName())) { Thread.sleep(100); } @@ -647,13 +648,13 @@ public class TestRegionObserverInterface { LOG.info(TestRegionObserverInterface.class.getName() +".testLegacyRecovery"); TableName tableName = TableName.valueOf(TEST_TABLE.getNameAsString() + ".testLegacyRecovery"); HTable table = util.createTable(tableName, new byte[][] {A, B, C}); - try { + try(RegionLocator r = util.getConnection().getRegionLocator(tableName)) { JVMClusterUtil.RegionServerThread rs1 = cluster.startRegionServer(); ServerName sn2 = rs1.getRegionServer().getServerName(); - String regEN = table.getRegionLocations().firstEntry().getKey().getEncodedName(); + String regEN = r.getAllRegionLocations().get(0).getRegionInfo().getEncodedName(); util.getHBaseAdmin().move(regEN.getBytes(), sn2.getServerName().getBytes()); - while (!sn2.equals(table.getRegionLocations().firstEntry().getValue() )){ + while (!sn2.equals(r.getAllRegionLocations().get(0).getServerName() )){ Thread.sleep(100); } @@ -700,11 +701,13 @@ public class TestRegionObserverInterface { JVMClusterUtil.RegionServerThread rs1 = cluster.startRegionServer(); ServerName sn2 = rs1.getRegionServer().getServerName(); - String regEN = table.getRegionLocations().firstEntry().getKey().getEncodedName(); + try (RegionLocator r = util.getConnection().getRegionLocator(tableName)) { + String regEN = r.getAllRegionLocations().get(0).getRegionInfo().getEncodedName(); - util.getHBaseAdmin().move(regEN.getBytes(), sn2.getServerName().getBytes()); - while (!sn2.equals(table.getRegionLocations().firstEntry().getValue())) { - Thread.sleep(100); + util.getHBaseAdmin().move(regEN.getBytes(), sn2.getServerName().getBytes()); + while (!sn2.equals(r.getAllRegionLocations().get(0).getServerName())) { + Thread.sleep(100); + } } Put put = new Put(ROW); diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/io/encoding/TestLoadAndSwitchEncodeOnDisk.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/io/encoding/TestLoadAndSwitchEncodeOnDisk.java index c648a8b..934f48f 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/io/encoding/TestLoadAndSwitchEncodeOnDisk.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/io/encoding/TestLoadAndSwitchEncodeOnDisk.java @@ -19,22 +19,21 @@ package org.apache.hadoop.hbase.io.encoding; import java.io.IOException; import java.util.Arrays; import java.util.Collection; -import java.util.Map; -import java.util.NavigableMap; +import java.util.List; import org.apache.hadoop.hbase.HColumnDescriptor; -import org.apache.hadoop.hbase.HRegionInfo; -import org.apache.hadoop.hbase.testclassification.IOTests; -import org.apache.hadoop.hbase.testclassification.MediumTests; -import org.apache.hadoop.hbase.ServerName; +import org.apache.hadoop.hbase.HRegionLocation; import org.apache.hadoop.hbase.client.HBaseAdmin; import org.apache.hadoop.hbase.client.HTable; +import org.apache.hadoop.hbase.client.RegionLocator; import org.apache.hadoop.hbase.client.Result; import org.apache.hadoop.hbase.client.ResultScanner; import org.apache.hadoop.hbase.client.Scan; import org.apache.hadoop.hbase.io.compress.Compression; import org.apache.hadoop.hbase.io.hfile.CacheConfig; import org.apache.hadoop.hbase.regionserver.HRegionServer; +import org.apache.hadoop.hbase.testclassification.IOTests; +import org.apache.hadoop.hbase.testclassification.MediumTests; import org.apache.hadoop.hbase.util.TestMiniClusterLoadSequential; import org.apache.hadoop.hbase.util.Threads; import org.junit.Test; @@ -104,14 +103,16 @@ public class TestLoadAndSwitchEncodeOnDisk extends } private void assertAllOnLine(final HTable t) throws IOException { - NavigableMap regions = t.getRegionLocations(); - for (Map.Entry e: regions.entrySet()) { - byte [] startkey = e.getKey().getStartKey(); - Scan s = new Scan(startkey); - ResultScanner scanner = t.getScanner(s); - Result r = scanner.next(); - org.junit.Assert.assertTrue(r != null && r.size() > 0); - scanner.close(); + try (RegionLocator rl = TEST_UTIL.getConnection().getRegionLocator(t.getName())) { + List regions = rl.getAllRegionLocations(); + for (HRegionLocation e : regions) { + byte[] startkey = e.getRegionInfo().getStartKey(); + Scan s = new Scan(startkey); + ResultScanner scanner = t.getScanner(s); + Result r = scanner.next(); + org.junit.Assert.assertTrue(r != null && r.size() > 0); + scanner.close(); + } } } } diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/mapreduce/TestHFileOutputFormat.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/mapreduce/TestHFileOutputFormat.java index b3c29b7..2dfb294 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/mapreduce/TestHFileOutputFormat.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/mapreduce/TestHFileOutputFormat.java @@ -429,10 +429,12 @@ public class TestHFileOutputFormat { util.deleteTable(table.getName()); byte[][] newSplitKeys = generateRandomSplitKeys(14); table = util.createTable(TABLE_NAME, FAMILIES, newSplitKeys); - while (table.getRegionLocations().size() != 15 || - !admin.isTableAvailable(table.getName())) { - Thread.sleep(200); - LOG.info("Waiting for new region assignment to happen"); + try (RegionLocator r = util.getConnection().getRegionLocator(TABLE_NAME)) { + while (r.getAllRegionLocations().size() != 15 || + !admin.isTableAvailable(table.getName())) { + Thread.sleep(200); + LOG.info("Waiting for new region assignment to happen"); + } } } diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestRegionPlacement.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestRegionPlacement.java index 9cbf680..6e89ff4 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestRegionPlacement.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestRegionPlacement.java @@ -41,6 +41,7 @@ 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.HRegionLocation; import org.apache.hadoop.hbase.HTableDescriptor; import org.apache.hadoop.hbase.MetaTableAccessor; import org.apache.hadoop.hbase.MiniHBaseCluster; @@ -50,6 +51,7 @@ import org.apache.hadoop.hbase.TableName; import org.apache.hadoop.hbase.client.Admin; import org.apache.hadoop.hbase.client.Connection; import org.apache.hadoop.hbase.client.HTable; +import org.apache.hadoop.hbase.client.RegionLocator; import org.apache.hadoop.hbase.client.Result; import org.apache.hadoop.hbase.master.balancer.FavoredNodeAssignmentHelper; import org.apache.hadoop.hbase.master.balancer.FavoredNodeLoadBalancer; @@ -158,7 +160,7 @@ public class TestRegionPlacement { // Check when a RS stops, the regions get assigned to their secondary/tertiary killRandomServerAndVerifyAssignment(); - + // also verify that the AssignmentVerificationReport has the correct information reports = rp.verifyRegionPlacement(false); report = reports.get(0); @@ -177,7 +179,7 @@ public class TestRegionPlacement { RegionPlacementMaintainer.printAssignmentPlan(currentPlan); } - private void killRandomServerAndVerifyAssignment() + private void killRandomServerAndVerifyAssignment() throws IOException, InterruptedException, KeeperException { ServerName serverToKill = null; int killIndex = 0; @@ -547,10 +549,12 @@ public class TestRegionPlacement { admin.createTable(desc, splitKeys); HTable ht = (HTable) CONNECTION.getTable(tableName); - @SuppressWarnings("deprecation") - Map regions = ht.getRegionLocations(); - assertEquals("Tried to create " + expectedRegions + " regions " - + "but only found " + regions.size(), expectedRegions, regions.size()); + try (RegionLocator r = CONNECTION.getRegionLocator(tableName)) { + List regions = r.getAllRegionLocations(); + assertEquals( + "Tried to create " + expectedRegions + " regions " + "but only found " + regions.size(), + expectedRegions, regions.size()); + } ht.close(); } } diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestHRegionOnCluster.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestHRegionOnCluster.java index 9b8dabf..d49a8ee 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestHRegionOnCluster.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestHRegionOnCluster.java @@ -17,7 +17,9 @@ */ package org.apache.hadoop.hbase.regionserver; -import static org.junit.Assert.*; +import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertFalse; +import static org.junit.Assert.assertTrue; import java.io.IOException; import java.util.ArrayList; @@ -30,8 +32,11 @@ import org.apache.hadoop.hbase.HTableDescriptor; import org.apache.hadoop.hbase.MiniHBaseCluster; import org.apache.hadoop.hbase.TableName; import org.apache.hadoop.hbase.client.Admin; +import org.apache.hadoop.hbase.client.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; import org.apache.hadoop.hbase.client.ResultScanner; import org.apache.hadoop.hbase.client.Scan; @@ -58,6 +63,8 @@ public class TestHRegionOnCluster { final int NUM_MASTERS = 1; final int NUM_RS = 3; Admin hbaseAdmin = null; + Connection conn = null; + RegionLocator r = null; TEST_UTIL.startMiniCluster(NUM_MASTERS, NUM_RS); try { @@ -81,7 +88,9 @@ public class TestHRegionOnCluster { TEST_UTIL.waitUntilAllRegionsAssigned(table.getName()); // Move region to target server - HRegionInfo regionInfo = table.getRegionLocation("r1").getRegionInfo(); + conn = ConnectionFactory.createConnection(TEST_UTIL.getConfiguration()); + r = conn.getRegionLocator(TABLENAME); + HRegionInfo regionInfo = r.getRegionLocation(Bytes.toBytes("r1")).getRegionInfo(); int originServerNum = cluster.getServerWith(regionInfo.getRegionName()); HRegionServer originServer = cluster.getRegionServer(originServerNum); int targetServerNum = (originServerNum + 1) % NUM_RS; @@ -91,7 +100,7 @@ public class TestHRegionOnCluster { TEST_UTIL.waitUntilAllRegionsAssigned(table.getName()); Log.info("Moving " + regionInfo.getEncodedName() + " to " + targetServer.getServerName()); hbaseAdmin.move(regionInfo.getEncodedNameAsBytes(), - Bytes.toBytes(targetServer.getServerName().getServerName())); + Bytes.toBytes(targetServer.getServerName().getServerName())); do { Thread.sleep(1); } while (cluster.getServerWith(regionInfo.getRegionName()) == originServerNum); @@ -104,7 +113,7 @@ public class TestHRegionOnCluster { // Move region to origin server Log.info("Moving " + regionInfo.getEncodedName() + " to " + originServer.getServerName()); hbaseAdmin.move(regionInfo.getEncodedNameAsBytes(), - Bytes.toBytes(originServer.getServerName().getServerName())); + Bytes.toBytes(originServer.getServerName().getServerName())); do { Thread.sleep(1); } while (cluster.getServerWith(regionInfo.getRegionName()) == targetServerNum); @@ -129,9 +138,14 @@ public class TestHRegionOnCluster { // Put data: r4->v4 Log.info("Loading r4 to v4 into " + TABLENAME); putDataAndVerify(table, "r4", FAMILY, "v4", 4); - } finally { if (hbaseAdmin != null) hbaseAdmin.close(); + if (r != null) { + r.close(); + } + if (conn != null) { + conn.close(); + } TEST_UTIL.shutdownMiniCluster(); } } diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestRegionReplicas.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestRegionReplicas.java index 85a8cd2..66e10e4 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestRegionReplicas.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestRegionReplicas.java @@ -18,7 +18,9 @@ package org.apache.hadoop.hbase.regionserver; -import static org.apache.hadoop.hbase.regionserver.TestRegionServerNoMaster.*; +import static org.apache.hadoop.hbase.regionserver.TestRegionServerNoMaster.closeRegion; +import static org.apache.hadoop.hbase.regionserver.TestRegionServerNoMaster.openRegion; + import java.io.IOException; import java.util.Random; import java.util.concurrent.ExecutorService; @@ -29,24 +31,26 @@ import java.util.concurrent.atomic.AtomicReference; import org.apache.commons.logging.Log; import org.apache.commons.logging.LogFactory; +import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.hbase.Cell; import org.apache.hadoop.hbase.HBaseTestingUtility; import org.apache.hadoop.hbase.HConstants; import org.apache.hadoop.hbase.HRegionInfo; -import org.apache.hadoop.hbase.testclassification.MediumTests; -import org.apache.hadoop.hbase.testclassification.RegionServerTests; import org.apache.hadoop.hbase.TableName; import org.apache.hadoop.hbase.TestMetaTableAccessor; import org.apache.hadoop.hbase.client.Consistency; import org.apache.hadoop.hbase.client.Get; import org.apache.hadoop.hbase.client.HTable; import org.apache.hadoop.hbase.client.Put; +import org.apache.hadoop.hbase.client.RegionLocator; import org.apache.hadoop.hbase.client.Result; import org.apache.hadoop.hbase.client.Table; import org.apache.hadoop.hbase.io.hfile.HFileScanner; import org.apache.hadoop.hbase.protobuf.ProtobufUtil; import org.apache.hadoop.hbase.protobuf.RequestConverter; import org.apache.hadoop.hbase.protobuf.generated.ClientProtos; +import org.apache.hadoop.hbase.testclassification.MediumTests; +import org.apache.hadoop.hbase.testclassification.RegionServerTests; import org.apache.hadoop.hbase.util.Bytes; import org.apache.hadoop.hbase.util.Threads; import org.apache.hadoop.hdfs.DFSConfigKeys; @@ -81,9 +85,10 @@ public class TestRegionReplicas { public static void before() throws Exception { // Reduce the hdfs block size and prefetch to trigger the file-link reopen // when the file is moved to archive (e.g. compaction) - HTU.getConfiguration().setInt(DFSConfigKeys.DFS_BLOCK_SIZE_KEY, 8192); - HTU.getConfiguration().setInt(DFSConfigKeys.DFS_CLIENT_READ_PREFETCH_SIZE_KEY, 1); - HTU.getConfiguration().setInt(HConstants.HREGION_MEMSTORE_FLUSH_SIZE, 128 * 1024 * 1024); + Configuration conf = HTU.getConfiguration(); + conf.setInt(DFSConfigKeys.DFS_BLOCK_SIZE_KEY, 8192); + conf.setInt(DFSConfigKeys.DFS_CLIENT_READ_PREFETCH_SIZE_KEY, 1); + conf.setInt(HConstants.HREGION_MEMSTORE_FLUSH_SIZE, 128 * 1024 * 1024); HTU.startMiniCluster(NB_SERVERS); final TableName tableName = TableName.valueOf(TestRegionReplicas.class.getSimpleName()); @@ -91,7 +96,9 @@ public class TestRegionReplicas { // Create table then get the single region for our new table. table = HTU.createTable(tableName, f); - hriPrimary = table.getRegionLocation(row, false).getRegionInfo(); + try (RegionLocator r = HTU.getConnection().getRegionLocator(tableName)) { + hriPrimary = r.getRegionLocation(row, false).getRegionInfo(); + } // mock a secondary region info to open hriSecondary = new HRegionInfo(hriPrimary.getTable(), hriPrimary.getStartKey(), diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestRegionServerNoMaster.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestRegionServerNoMaster.java index d150e01..edd8867 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestRegionServerNoMaster.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestRegionServerNoMaster.java @@ -30,6 +30,7 @@ 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.client.RegionLocator; import org.apache.hadoop.hbase.master.HMaster; import org.apache.hadoop.hbase.protobuf.ProtobufUtil; import org.apache.hadoop.hbase.protobuf.RequestConverter; @@ -78,7 +79,9 @@ public class TestRegionServerNoMaster { p.add(HConstants.CATALOG_FAMILY, row, row); table.put(p); - hri = table.getRegionLocation(row, false).getRegionInfo(); + try (RegionLocator r = HTU.getConnection().getRegionLocator(tableName)) { + hri = r.getRegionLocation(row, false).getRegionInfo(); + } regionName = hri.getRegionName(); stopMasterAndAssignMeta(HTU); diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestRegionServerOnlineConfigChange.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestRegionServerOnlineConfigChange.java index 9c6ee1a..c55a5df 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestRegionServerOnlineConfigChange.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestRegionServerOnlineConfigChange.java @@ -22,13 +22,17 @@ import static org.junit.Assert.assertEquals; import static org.junit.Assert.assertFalse; import static org.junit.Assert.assertTrue; +import java.io.IOException; + import org.apache.commons.logging.Log; import org.apache.commons.logging.LogFactory; import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.hbase.HBaseTestingUtility; import org.apache.hadoop.hbase.HConstants; import org.apache.hadoop.hbase.HRegionInfo; +import org.apache.hadoop.hbase.TableName; import org.apache.hadoop.hbase.client.HTable; +import org.apache.hadoop.hbase.client.RegionLocator; import org.apache.hadoop.hbase.regionserver.compactions.CompactionConfiguration; import org.apache.hadoop.hbase.testclassification.MediumTests; import org.apache.hadoop.hbase.util.Bytes; @@ -37,8 +41,6 @@ import org.junit.BeforeClass; import org.junit.Test; import org.junit.experimental.categories.Category; -import java.io.IOException; - /** * Verify that the Online config Changes on the HRegionServer side are actually * happening. We should add tests for important configurations which will be @@ -68,9 +70,11 @@ public class TestRegionServerOnlineConfigChange { conf = hbaseTestingUtility.getConfiguration(); hbaseTestingUtility.startMiniCluster(1,1); t1 = hbaseTestingUtility.createTable(TABLE1, COLUMN_FAMILY1); - @SuppressWarnings("deprecation") - HRegionInfo firstHRI = t1.getRegionLocations().keySet().iterator().next(); - r1name = firstHRI.getRegionName(); + try (RegionLocator r = + hbaseTestingUtility.getConnection().getRegionLocator(TableName.valueOf(TABLE1))) { + HRegionInfo firstHRI = r.getAllRegionLocations().iterator().next().getRegionInfo(); + r1name = firstHRI.getRegionName(); + } rs1 = hbaseTestingUtility.getHBaseCluster().getRegionServer( hbaseTestingUtility.getHBaseCluster().getServerWith(r1name)); r1 = rs1.getRegion(r1name); @@ -123,7 +127,7 @@ public class TestRegionServerOnlineConfigChange { HStore hstore = (HStore)s; // Set the new compaction ratio to a different value. - double newCompactionRatio = + double newCompactionRatio = hstore.getStoreEngine().getCompactionPolicy().getConf().getCompactionRatio() + 0.1; conf.setFloat(strPrefix + "ratio", (float)newCompactionRatio); diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestServerCustomProtocol.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestServerCustomProtocol.java index 5ce4456..0ea3505 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestServerCustomProtocol.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestServerCustomProtocol.java @@ -24,16 +24,13 @@ import static org.junit.Assert.assertTrue; import java.io.IOException; import java.util.Map; -import java.util.Map.Entry; import org.apache.commons.logging.Log; import org.apache.commons.logging.LogFactory; import org.apache.hadoop.hbase.Coprocessor; import org.apache.hadoop.hbase.CoprocessorEnvironment; import org.apache.hadoop.hbase.HBaseTestingUtility; -import org.apache.hadoop.hbase.HRegionInfo; import org.apache.hadoop.hbase.HRegionLocation; -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; @@ -323,7 +320,7 @@ public class TestServerCustomProtocol { // rows from 1 region assertEquals(1, results.size()); verifyRegionResults(locator, results, ROW_A); - + final String name = "NAME"; results = hello(table, name, null, ROW_A); // Should have gotten results for 1 of the three regions only since we specified @@ -337,59 +334,60 @@ public class TestServerCustomProtocol { public void testRowRange() throws Throwable { try (HTable table = (HTable) util.getConnection().getTable(TEST_TABLE)) { RegionLocator locator = table.getRegionLocator(); - for (Entry e: table.getRegionLocations().entrySet()) { - LOG.info("Region " + e.getKey().getRegionNameAsString() + ", servername=" + e.getValue()); + for (HRegionLocation e : locator.getAllRegionLocations()) { + LOG.info("Region " + e.getRegionInfo().getRegionNameAsString() + ", servername=" + + e.getServerName()); } // Here are what regions looked like on a run: // // test,,1355943549657.c65d4822d8bdecc033a96451f3a0f55d. // test,bbb,1355943549661.110393b070dd1ed93441e0bc9b3ffb7e. // test,ccc,1355943549665.c3d6d125141359cbbd2a43eaff3cdf74. - + Map results = ping(table, null, ROW_A); // Should contain first region only. assertEquals(1, results.size()); verifyRegionResults(locator, results, ROW_A); - + // Test start row + empty end results = ping(table, ROW_BC, null); assertEquals(2, results.size()); // should contain last 2 regions - HRegionLocation loc = table.getRegionLocation(ROW_A, true); + HRegionLocation loc = locator.getRegionLocation(ROW_A, true); assertNull("Should be missing region for row aaa (prior to start row)", results.get(loc.getRegionInfo().getRegionName())); verifyRegionResults(locator, results, ROW_B); verifyRegionResults(locator, results, ROW_C); - + // test empty start + end results = ping(table, null, ROW_BC); // should contain the first 2 regions assertEquals(2, results.size()); verifyRegionResults(locator, results, ROW_A); verifyRegionResults(locator, results, ROW_B); - loc = table.getRegionLocation(ROW_C, true); + loc = locator.getRegionLocation(ROW_C, true); assertNull("Should be missing region for row ccc (past stop row)", results.get(loc.getRegionInfo().getRegionName())); - + // test explicit start + end results = ping(table, ROW_AB, ROW_BC); // should contain first 2 regions assertEquals(2, results.size()); verifyRegionResults(locator, results, ROW_A); verifyRegionResults(locator, results, ROW_B); - loc = table.getRegionLocation(ROW_C, true); + loc = locator.getRegionLocation(ROW_C, true); assertNull("Should be missing region for row ccc (past stop row)", results.get(loc.getRegionInfo().getRegionName())); - + // test single region results = ping(table, ROW_B, ROW_BC); // should only contain region bbb assertEquals(1, results.size()); verifyRegionResults(locator, results, ROW_B); - loc = table.getRegionLocation(ROW_A, true); + loc = locator.getRegionLocation(ROW_A, true); assertNull("Should be missing region for row aaa (prior to start)", results.get(loc.getRegionInfo().getRegionName())); - loc = table.getRegionLocation(ROW_C, true); + loc = locator.getRegionLocation(ROW_C, true); assertNull("Should be missing region for row ccc (past stop row)", results.get(loc.getRegionInfo().getRegionName())); } diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/regionserver/TestRegionReplicaReplicationEndpointNoMaster.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/regionserver/TestRegionReplicaReplicationEndpointNoMaster.java index 930ffba..fb20734 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/regionserver/TestRegionReplicaReplicationEndpointNoMaster.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/regionserver/TestRegionReplicaReplicationEndpointNoMaster.java @@ -19,7 +19,7 @@ package org.apache.hadoop.hbase.replication.regionserver; import static org.apache.hadoop.hbase.regionserver.TestRegionServerNoMaster.closeRegion; import static org.apache.hadoop.hbase.regionserver.TestRegionServerNoMaster.openRegion; -import static org.junit.Assert.*; +import static org.junit.Assert.assertEquals; import static org.mockito.Mockito.mock; import static org.mockito.Mockito.when; @@ -40,6 +40,7 @@ import org.apache.hadoop.hbase.TableName; import org.apache.hadoop.hbase.client.ClusterConnection; 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.RpcRetryingCallerFactory; import org.apache.hadoop.hbase.coprocessor.BaseWALObserver; import org.apache.hadoop.hbase.coprocessor.CoprocessorHost; @@ -47,12 +48,9 @@ import org.apache.hadoop.hbase.coprocessor.ObserverContext; import org.apache.hadoop.hbase.coprocessor.WALCoprocessorEnvironment; import org.apache.hadoop.hbase.ipc.RpcControllerFactory; import org.apache.hadoop.hbase.protobuf.generated.AdminProtos.ReplicateWALEntryResponse; -import org.apache.hadoop.hbase.regionserver.HRegion; import org.apache.hadoop.hbase.regionserver.HRegionServer; import org.apache.hadoop.hbase.regionserver.Region; import org.apache.hadoop.hbase.regionserver.TestRegionServerNoMaster; -import org.apache.hadoop.hbase.wal.WAL.Entry; -import org.apache.hadoop.hbase.wal.WALKey; import org.apache.hadoop.hbase.regionserver.wal.WALEdit; import org.apache.hadoop.hbase.replication.ReplicationEndpoint; import org.apache.hadoop.hbase.replication.ReplicationEndpoint.ReplicateContext; @@ -63,6 +61,8 @@ import org.apache.hadoop.hbase.testclassification.MediumTests; import org.apache.hadoop.hbase.testclassification.ReplicationTests; import org.apache.hadoop.hbase.util.Bytes; import org.apache.hadoop.hbase.util.ServerRegionReplicaUtil; +import org.apache.hadoop.hbase.wal.WAL.Entry; +import org.apache.hadoop.hbase.wal.WALKey; import org.junit.After; import org.junit.AfterClass; import org.junit.Assert; @@ -112,15 +112,17 @@ public class TestRegionReplicaReplicationEndpointNoMaster { } else { walCoprocs += "," + WALEditCopro.class.getName(); } - HTU.getConfiguration().set(CoprocessorHost.WAL_COPROCESSOR_CONF_KEY, + conf.set(CoprocessorHost.WAL_COPROCESSOR_CONF_KEY, walCoprocs); HTU.startMiniCluster(NB_SERVERS); // Create table then get the single region for our new table. HTableDescriptor htd = HTU.createTableDescriptor(tableName.toString()); - table = HTU.createTable(htd, new byte[][]{f}, HTU.getConfiguration()); + table = HTU.createTable(htd, new byte[][]{f}, conf); - hriPrimary = table.getRegionLocation(row, false).getRegionInfo(); + try (RegionLocator r = HTU.getConnection().getRegionLocator(tableName)) { + hriPrimary = r.getRegionLocation(row, false).getRegionInfo(); + } // mock a secondary region info to open hriSecondary = new HRegionInfo(hriPrimary.getTable(), hriPrimary.getStartKey(), diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/util/TestHBaseFsck.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/util/TestHBaseFsck.java index 0ff87af..dff6e84 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/util/TestHBaseFsck.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/util/TestHBaseFsck.java @@ -64,10 +64,10 @@ import org.apache.hadoop.hbase.HConstants; import org.apache.hadoop.hbase.HRegionInfo; import org.apache.hadoop.hbase.HRegionLocation; import org.apache.hadoop.hbase.HTableDescriptor; +import org.apache.hadoop.hbase.MetaTableAccessor; import org.apache.hadoop.hbase.MiniHBaseCluster; import org.apache.hadoop.hbase.ServerName; import org.apache.hadoop.hbase.TableName; -import org.apache.hadoop.hbase.MetaTableAccessor; import org.apache.hadoop.hbase.client.Admin; import org.apache.hadoop.hbase.client.ClusterConnection; import org.apache.hadoop.hbase.client.Connection; @@ -79,6 +79,7 @@ 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.RegionReplicaUtil; import org.apache.hadoop.hbase.client.Result; import org.apache.hadoop.hbase.client.ResultScanner; @@ -343,53 +344,53 @@ public class TestHBaseFsck { throws IOException, InterruptedException { LOG.info("** Before delete:"); dumpMeta(htd.getTableName()); + try (RegionLocator r = TEST_UTIL.getConnection().getRegionLocator(tbl.getName())) { + List locations = r.getAllRegionLocations(); + for (HRegionLocation location : locations) { + HRegionInfo hri = location.getRegionInfo(); + ServerName hsa = location.getServerName(); + if (Bytes.compareTo(hri.getStartKey(), startKey) == 0 + && Bytes.compareTo(hri.getEndKey(), endKey) == 0 && hri.getReplicaId() == replicaId) { + + LOG.info("RegionName: " + hri.getRegionNameAsString()); + byte[] deleteRow = hri.getRegionName(); + + if (unassign) { + LOG.info("Undeploying region " + hri + " from server " + hsa); + undeployRegion(connection, hsa, hri); + } - List locations = tbl.getAllRegionLocations(); - for (HRegionLocation location : locations) { - HRegionInfo hri = location.getRegionInfo(); - ServerName hsa = location.getServerName(); - if (Bytes.compareTo(hri.getStartKey(), startKey) == 0 - && Bytes.compareTo(hri.getEndKey(), endKey) == 0 - && hri.getReplicaId() == replicaId) { - - LOG.info("RegionName: " +hri.getRegionNameAsString()); - byte[] deleteRow = hri.getRegionName(); - - if (unassign) { - LOG.info("Undeploying region " + hri + " from server " + hsa); - undeployRegion(connection, hsa, hri); - } - - if (regionInfoOnly) { - LOG.info("deleting hdfs .regioninfo data: " + hri.toString() + hsa.toString()); - Path rootDir = FSUtils.getRootDir(conf); - FileSystem fs = rootDir.getFileSystem(conf); - Path p = new Path(FSUtils.getTableDir(rootDir, htd.getTableName()), - hri.getEncodedName()); - Path hriPath = new Path(p, HRegionFileSystem.REGION_INFO_FILE); - fs.delete(hriPath, true); - } + if (regionInfoOnly) { + LOG.info("deleting hdfs .regioninfo data: " + hri.toString() + hsa.toString()); + Path rootDir = FSUtils.getRootDir(conf); + FileSystem fs = rootDir.getFileSystem(conf); + Path p = + new Path(FSUtils.getTableDir(rootDir, htd.getTableName()), hri.getEncodedName()); + Path hriPath = new Path(p, HRegionFileSystem.REGION_INFO_FILE); + fs.delete(hriPath, true); + } - if (hdfs) { - LOG.info("deleting hdfs data: " + hri.toString() + hsa.toString()); - Path rootDir = FSUtils.getRootDir(conf); - FileSystem fs = rootDir.getFileSystem(conf); - Path p = new Path(FSUtils.getTableDir(rootDir, htd.getTableName()), - hri.getEncodedName()); - HBaseFsck.debugLsr(conf, p); - boolean success = fs.delete(p, true); - LOG.info("Deleted " + p + " sucessfully? " + success); - HBaseFsck.debugLsr(conf, p); - } + if (hdfs) { + LOG.info("deleting hdfs data: " + hri.toString() + hsa.toString()); + Path rootDir = FSUtils.getRootDir(conf); + FileSystem fs = rootDir.getFileSystem(conf); + Path p = + new Path(FSUtils.getTableDir(rootDir, htd.getTableName()), hri.getEncodedName()); + HBaseFsck.debugLsr(conf, p); + boolean success = fs.delete(p, true); + LOG.info("Deleted " + p + " sucessfully? " + success); + HBaseFsck.debugLsr(conf, p); + } - if (metaRow) { - try (Table meta = connection.getTable(TableName.META_TABLE_NAME, tableExecutorService)) { - Delete delete = new Delete(deleteRow); - meta.delete(delete); + if (metaRow) { + try (Table meta = connection.getTable(TableName.META_TABLE_NAME, tableExecutorService)) { + Delete delete = new Delete(deleteRow); + meta.delete(delete); + } } } + LOG.info(hri.toString() + hsa.toString()); } - LOG.info(hri.toString() + hsa.toString()); } TEST_UTIL.getMetaTableRows(htd.getTableName()); @@ -1666,16 +1667,21 @@ public class TestHBaseFsck { TableName table = TableName.valueOf("testLingeringSplitParent"); Table meta = null; + Connection conn = null; + RegionLocator r = null; try { setupTable(table); assertEquals(ROWKEYS.length, countRows()); // make sure data in regions, if in wal only there is no data loss admin.flush(table); - HRegionLocation location = tbl.getRegionLocation("B"); + conn = ConnectionFactory.createConnection(TEST_UTIL.getConfiguration()); + r = conn.getRegionLocator(table); + byte[] row = Bytes.toBytes("B"); + HRegionLocation location = r.getRegionLocation(row); // Delete one region from meta, but not hdfs, unassign it. - deleteRegion(conf, tbl.getTableDescriptor(), Bytes.toBytes("B"), + deleteRegion(conf, tbl.getTableDescriptor(), row, Bytes.toBytes("C"), true, true, false); // Create a new meta entry to fake it as a split parent. @@ -1683,7 +1689,7 @@ public class TestHBaseFsck { HRegionInfo hri = location.getRegionInfo(); HRegionInfo a = new HRegionInfo(tbl.getName(), - Bytes.toBytes("B"), Bytes.toBytes("BM")); + row, Bytes.toBytes("BM")); HRegionInfo b = new HRegionInfo(tbl.getName(), Bytes.toBytes("BM"), Bytes.toBytes("C")); @@ -1732,6 +1738,12 @@ public class TestHBaseFsck { assertNoErrors(doFsck(conf, false)); assertEquals(ROWKEYS.length, countRows()); } finally { + if (r != null) { + r.close(); + } + if (conn != null) { + conn.close(); + } cleanupTable(table); IOUtils.closeQuietly(meta); } @@ -1746,13 +1758,16 @@ public class TestHBaseFsck { TableName table = TableName.valueOf("testLingeringSplitParent"); Table meta = null; + Connection c = TEST_UTIL.getConnection(); + RegionLocator r = null; try { setupTable(table); assertEquals(ROWKEYS.length, countRows()); // make sure data in regions, if in wal only there is no data loss admin.flush(table); - HRegionLocation location = tbl.getRegionLocation(Bytes.toBytes("B")); + r = c.getRegionLocator(table); + HRegionLocation location = r.getRegionLocation(Bytes.toBytes("B")); meta = connection.getTable(TableName.META_TABLE_NAME, tableExecutorService); HRegionInfo hri = location.getRegionInfo(); @@ -1777,9 +1792,12 @@ public class TestHBaseFsck { assertEquals(ROWKEYS.length, countRows()); // assert that we still have the split regions - assertEquals(tbl.getStartKeys().length, SPLITS.length + 1 + 1); //SPLITS + 1 is # regions pre-split. + assertEquals(r.getStartKeys().length, SPLITS.length + 1 + 1); //SPLITS + 1 is # regions pre-split. assertNoErrors(doFsck(conf, false)); } finally { + if (r != null) { + r.close(); + } cleanupTable(table); IOUtils.closeQuietly(meta); } @@ -1793,13 +1811,17 @@ public class TestHBaseFsck { public void testSplitDaughtersNotInMeta() throws Exception { TableName table = TableName.valueOf("testSplitdaughtersNotInMeta"); Table meta = connection.getTable(TableName.META_TABLE_NAME, tableExecutorService); + Connection c = TEST_UTIL.getConnection(); + RegionLocator r = null; + try { setupTable(table); assertEquals(ROWKEYS.length, countRows()); // make sure data in regions, if in wal only there is no data loss admin.flush(table); - HRegionLocation location = tbl.getRegionLocation(Bytes.toBytes("B")); + r = c.getRegionLocator(table); + HRegionLocation location = r.getRegionLocation(Bytes.toBytes("B")); HRegionInfo hri = location.getRegionInfo(); @@ -1811,10 +1833,27 @@ public class TestHBaseFsck { PairOfSameType daughters = MetaTableAccessor.getDaughterRegions(meta.get(new Get(regionName))); - // Delete daughter regions from meta, but not hdfs, unassign it. - Map hris = tbl.getRegionLocations(); - undeployRegion(connection, hris.get(daughters.getFirst()), daughters.getFirst()); - undeployRegion(connection, hris.get(daughters.getSecond()), daughters.getSecond()); + try (RegionLocator rl = TEST_UTIL.getConnection().getRegionLocator(table)) { + // Delete daughter regions from meta, but not hdfs, unassign it. + List hris = rl.getAllRegionLocations(); + boolean firstFound = false; + boolean secondFound = false; + ServerName firstServerName = null; + ServerName secondServerName = null; + for (HRegionLocation hrl : hris) { + if (daughters.getFirst().equals(hrl.getRegionInfo())) { + firstServerName = hrl.getServerName(); + firstFound = true; + } else if (daughters.getSecond().equals(hrl.getRegionInfo())) { + secondServerName = hrl.getServerName(); + secondFound = true; + } else if (firstFound && secondFound) { + break; + } + } + undeployRegion(connection, firstServerName, daughters.getFirst()); + undeployRegion(connection, secondServerName, daughters.getSecond()); + } List deletes = new ArrayList<>(); deletes.add(new Delete(daughters.getFirst().getRegionName())); @@ -1847,10 +1886,13 @@ public class TestHBaseFsck { assertEquals(ROWKEYS.length, countRows()); // assert that we still have the split regions - assertEquals(tbl.getStartKeys().length, SPLITS.length + 1 + 1); //SPLITS + 1 is # regions pre-split. + assertEquals(r.getStartKeys().length, SPLITS.length + 1 + 1); //SPLITS + 1 is # regions pre-split. assertNoErrors(doFsck(conf, false)); //should be fixed by now } finally { meta.close(); + if (r != null) { + r.close(); + } cleanupTable(table); } } @@ -2650,27 +2692,29 @@ public class TestHBaseFsck { // make sure data in regions, if in wal only there is no data loss admin.flush(table); - HRegionInfo region1 = tbl.getRegionLocation(Bytes.toBytes("A")).getRegionInfo(); - HRegionInfo region2 = tbl.getRegionLocation(Bytes.toBytes("B")).getRegionInfo(); - int regionCountBeforeMerge = tbl.getRegionLocations().size(); + try (RegionLocator r = TEST_UTIL.getConnection().getRegionLocator(table)) { + HRegionInfo region1 = r.getRegionLocation(Bytes.toBytes("A")).getRegionInfo(); + HRegionInfo region2 = r.getRegionLocation(Bytes.toBytes("B")).getRegionInfo(); - assertNotEquals(region1, region2); + int regionCountBeforeMerge = r.getAllRegionLocations().size(); - // do a region merge - admin.mergeRegions(region1.getEncodedNameAsBytes(), - region2.getEncodedNameAsBytes(), false); + assertNotEquals(region1, region2); - // wait until region merged - long timeout = System.currentTimeMillis() + 30 * 1000; - while (true) { - if (tbl.getRegionLocations().size() < regionCountBeforeMerge) { - break; - } else if (System.currentTimeMillis() > timeout) { - fail("Time out waiting on region " + region1.getEncodedName() - + " and " + region2.getEncodedName() + " be merged"); + // do a region merge + admin.mergeRegions(region1.getEncodedNameAsBytes(), region2.getEncodedNameAsBytes(), false); + + // wait until region merged + long timeout = System.currentTimeMillis() + 30 * 1000; + while (true) { + if (r.getAllRegionLocations().size() < regionCountBeforeMerge) { + break; + } else if (System.currentTimeMillis() > timeout) { + fail("Time out waiting on region " + region1.getEncodedName() + " and " + + region2.getEncodedName() + " be merged"); + } + Thread.sleep(10); } - Thread.sleep(10); } assertEquals(ROWKEYS.length, countRows()); diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/util/TestRegionSplitter.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/util/TestRegionSplitter.java index 63154a8..0747b9b 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/util/TestRegionSplitter.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/util/TestRegionSplitter.java @@ -26,7 +26,6 @@ import static org.junit.Assert.assertTrue; import java.util.ArrayList; import java.util.List; -import java.util.Map; import org.apache.commons.lang.ArrayUtils; import org.apache.commons.logging.Log; @@ -34,9 +33,11 @@ import org.apache.commons.logging.LogFactory; import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.hbase.HBaseTestingUtility; import org.apache.hadoop.hbase.HRegionInfo; -import org.apache.hadoop.hbase.ServerName; +import org.apache.hadoop.hbase.HRegionLocation; import org.apache.hadoop.hbase.TableName; +import org.apache.hadoop.hbase.client.Connection; import org.apache.hadoop.hbase.client.HTable; +import org.apache.hadoop.hbase.client.RegionLocator; import org.apache.hadoop.hbase.testclassification.MediumTests; import org.apache.hadoop.hbase.testclassification.MiscTests; import org.apache.hadoop.hbase.util.RegionSplitter.HexStringSplit; @@ -307,30 +308,31 @@ public class TestRegionSplitter { verifyBounds(expectedBounds, tableName); } - private void verifyBounds(List expectedBounds, TableName tableName) - throws Exception { - // Get region boundaries from the cluster and verify their endpoints - final int numRegions = expectedBounds.size()-1; - final HTable hTable = (HTable) UTIL.getConnection().getTable(tableName); - final Map regionInfoMap = hTable.getRegionLocations(); - assertEquals(numRegions, regionInfoMap.size()); - for (Map.Entry entry: regionInfoMap.entrySet()) { - final HRegionInfo regionInfo = entry.getKey(); - byte[] regionStart = regionInfo.getStartKey(); - byte[] regionEnd = regionInfo.getEndKey(); - - // This region's start key should be one of the region boundaries - int startBoundaryIndex = indexOfBytes(expectedBounds, regionStart); - assertNotSame(-1, startBoundaryIndex); - - // This region's end key should be the region boundary that comes - // after the starting boundary. - byte[] expectedRegionEnd = expectedBounds.get( - startBoundaryIndex+1); - assertEquals(0, Bytes.compareTo(regionEnd, expectedRegionEnd)); - } - hTable.close(); + private void verifyBounds(List expectedBounds, TableName tableName) throws Exception { + // Get region boundaries from the cluster and verify their endpoints + final int numRegions = expectedBounds.size() - 1; + Connection connection = UTIL.getConnection(); + final HTable hTable = (HTable) connection.getTable(tableName); + try (RegionLocator r = connection.getRegionLocator(tableName)) { + final List regionInfoMap = r.getAllRegionLocations(); + assertEquals(numRegions, regionInfoMap.size()); + for (HRegionLocation entry : regionInfoMap) { + final HRegionInfo regionInfo = entry.getRegionInfo(); + byte[] regionStart = regionInfo.getStartKey(); + byte[] regionEnd = regionInfo.getEndKey(); + + // This region's start key should be one of the region boundaries + int startBoundaryIndex = indexOfBytes(expectedBounds, regionStart); + assertNotSame(-1, startBoundaryIndex); + + // This region's end key should be the region boundary that comes + // after the starting boundary. + byte[] expectedRegionEnd = expectedBounds.get(startBoundaryIndex + 1); + assertEquals(0, Bytes.compareTo(regionEnd, expectedRegionEnd)); + } } + hTable.close(); + } /** * List.indexOf() doesn't really work for a List, because byte[] diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/util/hbck/OfflineMetaRebuildTestCore.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/util/hbck/OfflineMetaRebuildTestCore.java index f7f1f43..81e4d11 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/util/hbck/OfflineMetaRebuildTestCore.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/util/hbck/OfflineMetaRebuildTestCore.java @@ -34,6 +34,7 @@ import org.apache.hadoop.fs.Path; import org.apache.hadoop.hbase.HBaseTestingUtility; import org.apache.hadoop.hbase.HColumnDescriptor; import org.apache.hadoop.hbase.HRegionInfo; +import org.apache.hadoop.hbase.HRegionLocation; import org.apache.hadoop.hbase.HTableDescriptor; import org.apache.hadoop.hbase.MetaTableAccessor; import org.apache.hadoop.hbase.NamespaceDescriptor; @@ -46,6 +47,7 @@ import org.apache.hadoop.hbase.client.Delete; import org.apache.hadoop.hbase.client.HBaseAdmin; import org.apache.hadoop.hbase.client.HTable; import org.apache.hadoop.hbase.client.Put; +import org.apache.hadoop.hbase.client.RegionLocator; import org.apache.hadoop.hbase.client.Result; import org.apache.hadoop.hbase.client.ResultScanner; import org.apache.hadoop.hbase.client.Scan; @@ -182,30 +184,31 @@ public class OfflineMetaRebuildTestCore { HTableDescriptor htd = tbl.getTableDescriptor(); dumpMeta(htd); - Map hris = ((HTable)tbl).getRegionLocations(); - for (Entry e : hris.entrySet()) { - HRegionInfo hri = e.getKey(); - ServerName hsa = e.getValue(); - if (Bytes.compareTo(hri.getStartKey(), startKey) == 0 - && Bytes.compareTo(hri.getEndKey(), endKey) == 0) { + try (RegionLocator r = TEST_UTIL.getConnection().getRegionLocator(tbl.getName())) { + List hris = r.getAllRegionLocations(); + for (HRegionLocation e : hris) { + HRegionInfo hri = e.getRegionInfo(); + ServerName hsa = e.getServerName(); + if (Bytes.compareTo(hri.getStartKey(), startKey) == 0 + && Bytes.compareTo(hri.getEndKey(), endKey) == 0) { - LOG.info("RegionName: " + hri.getRegionNameAsString()); - byte[] deleteRow = hri.getRegionName(); - TEST_UTIL.getHBaseAdmin().unassign(deleteRow, true); + LOG.info("RegionName: " + hri.getRegionNameAsString()); + byte[] deleteRow = hri.getRegionName(); + TEST_UTIL.getHBaseAdmin().unassign(deleteRow, true); - LOG.info("deleting hdfs data: " + hri.toString() + hsa.toString()); - Path rootDir = FSUtils.getRootDir(conf); - FileSystem fs = rootDir.getFileSystem(conf); - Path p = new Path(FSUtils.getTableDir(rootDir, htd.getTableName()), - hri.getEncodedName()); - fs.delete(p, true); + LOG.info("deleting hdfs data: " + hri.toString() + hsa.toString()); + Path rootDir = FSUtils.getRootDir(conf); + FileSystem fs = rootDir.getFileSystem(conf); + Path p = new Path(FSUtils.getTableDir(rootDir, htd.getTableName()), hri.getEncodedName()); + fs.delete(p, true); - try (Table meta = this.connection.getTable(TableName.META_TABLE_NAME)) { - Delete delete = new Delete(deleteRow); - meta.delete(delete); + try (Table meta = this.connection.getTable(TableName.META_TABLE_NAME)) { + Delete delete = new Delete(deleteRow); + meta.delete(delete); + } } + LOG.info(hri.toString() + hsa.toString()); } - LOG.info(hri.toString() + hsa.toString()); } TEST_UTIL.getMetaTableRows(htd.getTableName()); -- 1.9.2.msysgit.0
Region ServerRegion Count