HBASE-11995 Use Connection and ConnectionFactory where possible (Solomon Duskis)

This commit is contained in:
Enis Soztutar 2014-09-26 11:49:37 -07:00
parent 88cd708d8d
commit 3beb168b4f
25 changed files with 250 additions and 247 deletions

View File

@ -24,9 +24,9 @@ import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory; import org.apache.commons.logging.LogFactory;
import org.apache.hadoop.hbase.classification.InterfaceAudience; import org.apache.hadoop.hbase.classification.InterfaceAudience;
import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.hbase.client.Connection;
import org.apache.hadoop.hbase.client.Delete; import org.apache.hadoop.hbase.client.Delete;
import org.apache.hadoop.hbase.client.Get; import org.apache.hadoop.hbase.client.Get;
import org.apache.hadoop.hbase.client.HConnection;
import org.apache.hadoop.hbase.client.HTable; import org.apache.hadoop.hbase.client.HTable;
import org.apache.hadoop.hbase.client.Mutation; import org.apache.hadoop.hbase.client.Mutation;
import org.apache.hadoop.hbase.client.Put; import org.apache.hadoop.hbase.client.Put;
@ -133,65 +133,65 @@ public class MetaTableAccessor {
* @return List of {@link org.apache.hadoop.hbase.client.Result} * @return List of {@link org.apache.hadoop.hbase.client.Result}
* @throws IOException * @throws IOException
*/ */
public static List<Result> fullScanOfMeta(HConnection hConnection) public static List<Result> fullScanOfMeta(Connection connection)
throws IOException { throws IOException {
CollectAllVisitor v = new CollectAllVisitor(); CollectAllVisitor v = new CollectAllVisitor();
fullScan(hConnection, v, null); fullScan(connection, v, null);
return v.getResults(); return v.getResults();
} }
/** /**
* Performs a full scan of <code>hbase:meta</code>. * Performs a full scan of <code>hbase:meta</code>.
* @param hConnection connection we're using * @param connection connection we're using
* @param visitor Visitor invoked against each row. * @param visitor Visitor invoked against each row.
* @throws IOException * @throws IOException
*/ */
public static void fullScan(HConnection hConnection, public static void fullScan(Connection connection,
final Visitor visitor) final Visitor visitor)
throws IOException { throws IOException {
fullScan(hConnection, visitor, null); fullScan(connection, visitor, null);
} }
/** /**
* Performs a full scan of <code>hbase:meta</code>. * Performs a full scan of <code>hbase:meta</code>.
* @param hConnection connection we're using * @param connection connection we're using
* @return List of {@link Result} * @return List of {@link Result}
* @throws IOException * @throws IOException
*/ */
public static List<Result> fullScan(HConnection hConnection) public static List<Result> fullScan(Connection connection)
throws IOException { throws IOException {
CollectAllVisitor v = new CollectAllVisitor(); CollectAllVisitor v = new CollectAllVisitor();
fullScan(hConnection, v, null); fullScan(connection, v, null);
return v.getResults(); return v.getResults();
} }
/** /**
* Callers should call close on the returned {@link HTable} instance. * Callers should call close on the returned {@link HTable} instance.
* @param hConnection connection we're using to access table * @param connection connection we're using to access table
* @param tableName Table to get an {@link org.apache.hadoop.hbase.client.HTable} against. * @param tableName Table to get an {@link org.apache.hadoop.hbase.client.HTable} against.
* @return An {@link org.apache.hadoop.hbase.client.HTable} for <code>tableName</code> * @return An {@link org.apache.hadoop.hbase.client.HTable} for <code>tableName</code>
* @throws IOException * @throws IOException
* @SuppressWarnings("deprecation") * @SuppressWarnings("deprecation")
*/ */
private static Table getHTable(final HConnection hConnection, private static Table getHTable(final Connection connection,
final TableName tableName) final TableName tableName)
throws IOException { throws IOException {
// We used to pass whole CatalogTracker in here, now we just pass in HConnection // We used to pass whole CatalogTracker in here, now we just pass in Connection
if (hConnection == null || hConnection.isClosed()) { if (connection == null || connection.isClosed()) {
throw new NullPointerException("No connection"); throw new NullPointerException("No connection");
} }
return new HTable(tableName, hConnection); return new HTable(tableName, connection);
} }
/** /**
* Callers should call close on the returned {@link HTable} instance. * Callers should call close on the returned {@link HTable} instance.
* @param hConnection connection we're using to access Meta * @param connection connection we're using to access Meta
* @return An {@link HTable} for <code>hbase:meta</code> * @return An {@link HTable} for <code>hbase:meta</code>
* @throws IOException * @throws IOException
*/ */
static Table getMetaHTable(final HConnection hConnection) static Table getMetaHTable(final Connection connection)
throws IOException { throws IOException {
return getHTable(hConnection, TableName.META_TABLE_NAME); return getHTable(connection, TableName.META_TABLE_NAME);
} }
/** /**
@ -209,17 +209,17 @@ public class MetaTableAccessor {
/** /**
* Gets the region info and assignment for the specified region. * Gets the region info and assignment for the specified region.
* @param hConnection connection we're using * @param connection connection we're using
* @param regionName Region to lookup. * @param regionName Region to lookup.
* @return Location and HRegionInfo for <code>regionName</code> * @return Location and HRegionInfo for <code>regionName</code>
* @throws IOException * @throws IOException
* @deprecated use {@link #getRegionLocation(HConnection, byte[])} instead * @deprecated use {@link #getRegionLocation(Connection, byte[])} instead
*/ */
@Deprecated @Deprecated
public static Pair<HRegionInfo, ServerName> getRegion( public static Pair<HRegionInfo, ServerName> getRegion(
HConnection hConnection, byte [] regionName) Connection connection, byte [] regionName)
throws IOException { throws IOException {
HRegionLocation location = getRegionLocation(hConnection, regionName); HRegionLocation location = getRegionLocation(connection, regionName);
return location == null return location == null
? null ? null
: new Pair<HRegionInfo, ServerName>(location.getRegionInfo(), location.getServerName()); : new Pair<HRegionInfo, ServerName>(location.getRegionInfo(), location.getServerName());
@ -227,12 +227,12 @@ public class MetaTableAccessor {
/** /**
* Returns the HRegionLocation from meta for the given region * Returns the HRegionLocation from meta for the given region
* @param hConnection connection we're using * @param connection connection we're using
* @param regionName region we're looking for * @param regionName region we're looking for
* @return HRegionLocation for the given region * @return HRegionLocation for the given region
* @throws IOException * @throws IOException
*/ */
public static HRegionLocation getRegionLocation(HConnection hConnection, public static HRegionLocation getRegionLocation(Connection connection,
byte[] regionName) throws IOException { byte[] regionName) throws IOException {
byte[] row = regionName; byte[] row = regionName;
HRegionInfo parsedInfo = null; HRegionInfo parsedInfo = null;
@ -244,7 +244,7 @@ public class MetaTableAccessor {
} }
Get get = new Get(row); Get get = new Get(row);
get.addFamily(HConstants.CATALOG_FAMILY); get.addFamily(HConstants.CATALOG_FAMILY);
Result r = get(getMetaHTable(hConnection), get); Result r = get(getMetaHTable(connection), get);
RegionLocations locations = getRegionLocations(r); RegionLocations locations = getRegionLocations(r);
return locations == null return locations == null
? null ? null
@ -253,17 +253,17 @@ public class MetaTableAccessor {
/** /**
* Returns the HRegionLocation from meta for the given region * Returns the HRegionLocation from meta for the given region
* @param hConnection connection we're using * @param connection connection we're using
* @param regionInfo region information * @param regionInfo region information
* @return HRegionLocation for the given region * @return HRegionLocation for the given region
* @throws IOException * @throws IOException
*/ */
public static HRegionLocation getRegionLocation(HConnection hConnection, public static HRegionLocation getRegionLocation(Connection connection,
HRegionInfo regionInfo) throws IOException { HRegionInfo regionInfo) throws IOException {
byte[] row = getMetaKeyForRegion(regionInfo); byte[] row = getMetaKeyForRegion(regionInfo);
Get get = new Get(row); Get get = new Get(row);
get.addFamily(HConstants.CATALOG_FAMILY); get.addFamily(HConstants.CATALOG_FAMILY);
Result r = get(getMetaHTable(hConnection), get); Result r = get(getMetaHTable(connection), get);
return getRegionLocation(r, regionInfo, regionInfo.getReplicaId()); return getRegionLocation(r, regionInfo, regionInfo.getReplicaId());
} }
@ -287,16 +287,16 @@ public class MetaTableAccessor {
/** /**
* Gets the result in hbase:meta for the specified region. * Gets the result in hbase:meta for the specified region.
* @param hConnection connection we're using * @param connection connection we're using
* @param regionName region we're looking for * @param regionName region we're looking for
* @return result of the specified region * @return result of the specified region
* @throws IOException * @throws IOException
*/ */
public static Result getRegionResult(HConnection hConnection, public static Result getRegionResult(Connection connection,
byte[] regionName) throws IOException { byte[] regionName) throws IOException {
Get get = new Get(regionName); Get get = new Get(regionName);
get.addFamily(HConstants.CATALOG_FAMILY); get.addFamily(HConstants.CATALOG_FAMILY);
return get(getMetaHTable(hConnection), get); return get(getMetaHTable(connection), get);
} }
/** /**
@ -305,8 +305,8 @@ public class MetaTableAccessor {
* @throws IOException * @throws IOException
*/ */
public static Pair<HRegionInfo, HRegionInfo> getRegionsFromMergeQualifier( public static Pair<HRegionInfo, HRegionInfo> getRegionsFromMergeQualifier(
HConnection hConnection, byte[] regionName) throws IOException { Connection connection, byte[] regionName) throws IOException {
Result result = getRegionResult(hConnection, regionName); Result result = getRegionResult(connection, regionName);
HRegionInfo mergeA = getHRegionInfo(result, HConstants.MERGEA_QUALIFIER); HRegionInfo mergeA = getHRegionInfo(result, HConstants.MERGEA_QUALIFIER);
HRegionInfo mergeB = getHRegionInfo(result, HConstants.MERGEB_QUALIFIER); HRegionInfo mergeB = getHRegionInfo(result, HConstants.MERGEB_QUALIFIER);
if (mergeA == null && mergeB == null) { if (mergeA == null && mergeB == null) {
@ -318,12 +318,12 @@ public class MetaTableAccessor {
/** /**
* Checks if the specified table exists. Looks at the hbase:meta table hosted on * Checks if the specified table exists. Looks at the hbase:meta table hosted on
* the specified server. * the specified server.
* @param hConnection connection we're using * @param connection connection we're using
* @param tableName table to check * @param tableName table to check
* @return true if the table exists in meta, false if not * @return true if the table exists in meta, false if not
* @throws IOException * @throws IOException
*/ */
public static boolean tableExists(HConnection hConnection, public static boolean tableExists(Connection connection,
final TableName tableName) final TableName tableName)
throws IOException { throws IOException {
if (tableName.equals(TableName.META_TABLE_NAME)) { if (tableName.equals(TableName.META_TABLE_NAME)) {
@ -359,7 +359,7 @@ public class MetaTableAccessor {
this.results.add(this.current); this.results.add(this.current);
} }
}; };
fullScan(hConnection, visitor, getTableStartRowForMeta(tableName)); fullScan(connection, visitor, getTableStartRowForMeta(tableName));
// If visitor has results >= 1 then table exists. // If visitor has results >= 1 then table exists.
return visitor.getResults().size() >= 1; return visitor.getResults().size() >= 1;
} }
@ -367,32 +367,32 @@ public class MetaTableAccessor {
/** /**
* Gets all of the regions of the specified table. Do not use this method * Gets all of the regions of the specified table. Do not use this method
* to get meta table regions, use methods in MetaTableLocator instead. * to get meta table regions, use methods in MetaTableLocator instead.
* @param hConnection connection we're using * @param connection connection we're using
* @param tableName table we're looking for * @param tableName table we're looking for
* @return Ordered list of {@link HRegionInfo}. * @return Ordered list of {@link HRegionInfo}.
* @throws IOException * @throws IOException
*/ */
public static List<HRegionInfo> getTableRegions(HConnection hConnection, TableName tableName) public static List<HRegionInfo> getTableRegions(Connection connection, TableName tableName)
throws IOException { throws IOException {
return getTableRegions(hConnection, tableName, false); return getTableRegions(connection, tableName, false);
} }
/** /**
* Gets all of the regions of the specified table. Do not use this method * Gets all of the regions of the specified table. Do not use this method
* to get meta table regions, use methods in MetaTableLocator instead. * to get meta table regions, use methods in MetaTableLocator instead.
* @param hConnection connection we're using * @param connection connection we're using
* @param tableName table we're looking for * @param tableName table we're looking for
* @param excludeOfflinedSplitParents If true, do not include offlined split * @param excludeOfflinedSplitParents If true, do not include offlined split
* parents in the return. * parents in the return.
* @return Ordered list of {@link HRegionInfo}. * @return Ordered list of {@link HRegionInfo}.
* @throws IOException * @throws IOException
*/ */
public static List<HRegionInfo> getTableRegions(HConnection hConnection, public static List<HRegionInfo> getTableRegions(Connection connection,
TableName tableName, final boolean excludeOfflinedSplitParents) TableName tableName, final boolean excludeOfflinedSplitParents)
throws IOException { throws IOException {
List<Pair<HRegionInfo, ServerName>> result; List<Pair<HRegionInfo, ServerName>> result;
try { try {
result = getTableRegionsAndLocations(hConnection, tableName, result = getTableRegionsAndLocations(connection, tableName,
excludeOfflinedSplitParents); excludeOfflinedSplitParents);
} catch (InterruptedException e) { } catch (InterruptedException e) {
throw (InterruptedIOException)new InterruptedIOException().initCause(e); throw (InterruptedIOException)new InterruptedIOException().initCause(e);
@ -455,28 +455,28 @@ public class MetaTableAccessor {
/** /**
* Do not use this method to get meta table regions, use methods in MetaTableLocator instead. * Do not use this method to get meta table regions, use methods in MetaTableLocator instead.
* @param hConnection connection we're using * @param connection connection we're using
* @param tableName table we're looking for * @param tableName table we're looking for
* @return Return list of regioninfos and server. * @return Return list of regioninfos and server.
* @throws IOException * @throws IOException
* @throws InterruptedException * @throws InterruptedException
*/ */
public static List<Pair<HRegionInfo, ServerName>> public static List<Pair<HRegionInfo, ServerName>>
getTableRegionsAndLocations(HConnection hConnection, TableName tableName) getTableRegionsAndLocations(Connection connection, TableName tableName)
throws IOException, InterruptedException { throws IOException, InterruptedException {
return getTableRegionsAndLocations(hConnection, tableName, true); return getTableRegionsAndLocations(connection, tableName, true);
} }
/** /**
* Do not use this method to get meta table regions, use methods in MetaTableLocator instead. * Do not use this method to get meta table regions, use methods in MetaTableLocator instead.
* @param hConnection connection we're using * @param connection connection we're using
* @param tableName table to work with * @param tableName table to work with
* @return Return list of regioninfos and server addresses. * @return Return list of regioninfos and server addresses.
* @throws IOException * @throws IOException
* @throws InterruptedException * @throws InterruptedException
*/ */
public static List<Pair<HRegionInfo, ServerName>> getTableRegionsAndLocations( public static List<Pair<HRegionInfo, ServerName>> getTableRegionsAndLocations(
HConnection hConnection, final TableName tableName, Connection connection, final TableName tableName,
final boolean excludeOfflinedSplitParents) throws IOException, InterruptedException { final boolean excludeOfflinedSplitParents) throws IOException, InterruptedException {
if (tableName.equals(TableName.META_TABLE_NAME)) { if (tableName.equals(TableName.META_TABLE_NAME)) {
throw new IOException("This method can't be used to locate meta regions;" throw new IOException("This method can't be used to locate meta regions;"
@ -514,19 +514,19 @@ public class MetaTableAccessor {
} }
} }
}; };
fullScan(hConnection, visitor, getTableStartRowForMeta(tableName)); fullScan(connection, visitor, getTableStartRowForMeta(tableName));
return visitor.getResults(); return visitor.getResults();
} }
/** /**
* @param hConnection connection we're using * @param connection connection we're using
* @param serverName server whose regions we're interested in * @param serverName server whose regions we're interested in
* @return List of user regions installed on this server (does not include * @return List of user regions installed on this server (does not include
* catalog regions). * catalog regions).
* @throws IOException * @throws IOException
*/ */
public static NavigableMap<HRegionInfo, Result> public static NavigableMap<HRegionInfo, Result>
getServerUserRegions(HConnection hConnection, final ServerName serverName) getServerUserRegions(Connection connection, final ServerName serverName)
throws IOException { throws IOException {
final NavigableMap<HRegionInfo, Result> hris = new TreeMap<HRegionInfo, Result>(); final NavigableMap<HRegionInfo, Result> hris = new TreeMap<HRegionInfo, Result>();
// Fill the above hris map with entries from hbase:meta that have the passed // Fill the above hris map with entries from hbase:meta that have the passed
@ -546,11 +546,11 @@ public class MetaTableAccessor {
} }
} }
}; };
fullScan(hConnection, v); fullScan(connection, v);
return hris; return hris;
} }
public static void fullScanMetaAndPrint(HConnection hConnection) public static void fullScanMetaAndPrint(Connection connection)
throws IOException { throws IOException {
Visitor v = new Visitor() { Visitor v = new Visitor() {
@Override @Override
@ -567,30 +567,30 @@ public class MetaTableAccessor {
return true; return true;
} }
}; };
fullScan(hConnection, v); fullScan(connection, v);
} }
/** /**
* Performs a full scan of a catalog table. * Performs a full scan of a catalog table.
* @param hConnection connection we're using * @param connection connection we're using
* @param visitor Visitor invoked against each row. * @param visitor Visitor invoked against each row.
* @param startrow Where to start the scan. Pass null if want to begin scan * @param startrow Where to start the scan. Pass null if want to begin scan
* at first row. * at first row.
* <code>hbase:meta</code>, the default (pass false to scan hbase:meta) * <code>hbase:meta</code>, the default (pass false to scan hbase:meta)
* @throws IOException * @throws IOException
*/ */
public static void fullScan(HConnection hConnection, public static void fullScan(Connection connection,
final Visitor visitor, final byte [] startrow) final Visitor visitor, final byte [] startrow)
throws IOException { throws IOException {
Scan scan = new Scan(); Scan scan = new Scan();
if (startrow != null) scan.setStartRow(startrow); if (startrow != null) scan.setStartRow(startrow);
if (startrow == null) { if (startrow == null) {
int caching = hConnection.getConfiguration() int caching = connection.getConfiguration()
.getInt(HConstants.HBASE_META_SCANNER_CACHING, 100); .getInt(HConstants.HBASE_META_SCANNER_CACHING, 100);
scan.setCaching(caching); scan.setCaching(caching);
} }
scan.addFamily(HConstants.CATALOG_FAMILY); scan.addFamily(HConstants.CATALOG_FAMILY);
Table metaTable = getMetaHTable(hConnection); Table metaTable = getMetaHTable(connection);
ResultScanner scanner = null; ResultScanner scanner = null;
try { try {
scanner = metaTable.getScanner(scan); scanner = metaTable.getScanner(scan);
@ -925,13 +925,13 @@ public class MetaTableAccessor {
/** /**
* Put the passed <code>p</code> to the <code>hbase:meta</code> table. * Put the passed <code>p</code> to the <code>hbase:meta</code> table.
* @param hConnection connection we're using * @param connection connection we're using
* @param p Put to add to hbase:meta * @param p Put to add to hbase:meta
* @throws IOException * @throws IOException
*/ */
static void putToMetaTable(final HConnection hConnection, final Put p) static void putToMetaTable(final Connection connection, final Put p)
throws IOException { throws IOException {
put(getMetaHTable(hConnection), p); put(getMetaHTable(connection), p);
} }
/** /**
@ -949,13 +949,13 @@ public class MetaTableAccessor {
/** /**
* Put the passed <code>ps</code> to the <code>hbase:meta</code> table. * Put the passed <code>ps</code> to the <code>hbase:meta</code> table.
* @param hConnection connection we're using * @param connection connection we're using
* @param ps Put to add to hbase:meta * @param ps Put to add to hbase:meta
* @throws IOException * @throws IOException
*/ */
public static void putsToMetaTable(final HConnection hConnection, final List<Put> ps) public static void putsToMetaTable(final Connection connection, final List<Put> ps)
throws IOException { throws IOException {
Table t = getMetaHTable(hConnection); Table t = getMetaHTable(connection);
try { try {
t.put(ps); t.put(ps);
} finally { } finally {
@ -965,26 +965,26 @@ public class MetaTableAccessor {
/** /**
* Delete the passed <code>d</code> from the <code>hbase:meta</code> table. * Delete the passed <code>d</code> from the <code>hbase:meta</code> table.
* @param hConnection connection we're using * @param connection connection we're using
* @param d Delete to add to hbase:meta * @param d Delete to add to hbase:meta
* @throws IOException * @throws IOException
*/ */
static void deleteFromMetaTable(final HConnection hConnection, final Delete d) static void deleteFromMetaTable(final Connection connection, final Delete d)
throws IOException { throws IOException {
List<Delete> dels = new ArrayList<Delete>(1); List<Delete> dels = new ArrayList<Delete>(1);
dels.add(d); dels.add(d);
deleteFromMetaTable(hConnection, dels); deleteFromMetaTable(connection, dels);
} }
/** /**
* Delete the passed <code>deletes</code> from the <code>hbase:meta</code> table. * Delete the passed <code>deletes</code> from the <code>hbase:meta</code> table.
* @param hConnection connection we're using * @param connection connection we're using
* @param deletes Deletes to add to hbase:meta This list should support #remove. * @param deletes Deletes to add to hbase:meta This list should support #remove.
* @throws IOException * @throws IOException
*/ */
public static void deleteFromMetaTable(final HConnection hConnection, final List<Delete> deletes) public static void deleteFromMetaTable(final Connection connection, final List<Delete> deletes)
throws IOException { throws IOException {
Table t = getMetaHTable(hConnection); Table t = getMetaHTable(connection);
try { try {
t.delete(deletes); t.delete(deletes);
} finally { } finally {
@ -997,11 +997,11 @@ public class MetaTableAccessor {
* @param metaRows rows in hbase:meta * @param metaRows rows in hbase:meta
* @param replicaIndexToDeleteFrom the replica ID we would start deleting from * @param replicaIndexToDeleteFrom the replica ID we would start deleting from
* @param numReplicasToRemove how many replicas to remove * @param numReplicasToRemove how many replicas to remove
* @param hConnection connection we're using to access meta table * @param connection connection we're using to access meta table
* @throws IOException * @throws IOException
*/ */
public static void removeRegionReplicasFromMeta(Set<byte[]> metaRows, public static void removeRegionReplicasFromMeta(Set<byte[]> metaRows,
int replicaIndexToDeleteFrom, int numReplicasToRemove, HConnection hConnection) int replicaIndexToDeleteFrom, int numReplicasToRemove, Connection connection)
throws IOException { throws IOException {
int absoluteIndex = replicaIndexToDeleteFrom + numReplicasToRemove; int absoluteIndex = replicaIndexToDeleteFrom + numReplicasToRemove;
for (byte[] row : metaRows) { for (byte[] row : metaRows) {
@ -1014,20 +1014,20 @@ public class MetaTableAccessor {
deleteReplicaLocations.deleteColumns(HConstants.CATALOG_FAMILY, deleteReplicaLocations.deleteColumns(HConstants.CATALOG_FAMILY,
getStartCodeColumn(i)); getStartCodeColumn(i));
} }
deleteFromMetaTable(hConnection, deleteReplicaLocations); deleteFromMetaTable(connection, deleteReplicaLocations);
} }
} }
/** /**
* Execute the passed <code>mutations</code> against <code>hbase:meta</code> table. * Execute the passed <code>mutations</code> against <code>hbase:meta</code> table.
* @param hConnection connection we're using * @param connection connection we're using
* @param mutations Puts and Deletes to execute on hbase:meta * @param mutations Puts and Deletes to execute on hbase:meta
* @throws IOException * @throws IOException
*/ */
public static void mutateMetaTable(final HConnection hConnection, public static void mutateMetaTable(final Connection connection,
final List<Mutation> mutations) final List<Mutation> mutations)
throws IOException { throws IOException {
Table t = getMetaHTable(hConnection); Table t = getMetaHTable(connection);
try { try {
t.batch(mutations); t.batch(mutations);
} catch (InterruptedException e) { } catch (InterruptedException e) {
@ -1041,14 +1041,14 @@ public class MetaTableAccessor {
/** /**
* Adds a hbase:meta row for the specified new region. * Adds a hbase:meta row for the specified new region.
* @param hConnection connection we're using * @param connection connection we're using
* @param regionInfo region information * @param regionInfo region information
* @throws IOException if problem connecting or updating meta * @throws IOException if problem connecting or updating meta
*/ */
public static void addRegionToMeta(HConnection hConnection, public static void addRegionToMeta(Connection connection,
HRegionInfo regionInfo) HRegionInfo regionInfo)
throws IOException { throws IOException {
putToMetaTable(hConnection, makePutFromRegionInfo(regionInfo)); putToMetaTable(connection, makePutFromRegionInfo(regionInfo));
LOG.info("Added " + regionInfo.getRegionNameAsString()); LOG.info("Added " + regionInfo.getRegionNameAsString());
} }
@ -1067,7 +1067,7 @@ public class MetaTableAccessor {
* Adds a (single) hbase:meta row for the specified new region and its daughters. Note that this * Adds a (single) hbase:meta row for the specified new region and its daughters. Note that this
* does not add its daughter's as different rows, but adds information about the daughters * does not add its daughter's as different rows, but adds information about the daughters
* in the same row as the parent. Use * in the same row as the parent. Use
* {@link #splitRegion(org.apache.hadoop.hbase.client.HConnection, * {@link #splitRegion(org.apache.hadoop.hbase.client.Connection,
* HRegionInfo, HRegionInfo, HRegionInfo, ServerName)} * HRegionInfo, HRegionInfo, HRegionInfo, ServerName)}
* if you want to do that. * if you want to do that.
* @param meta the HTable for META * @param meta the HTable for META
@ -1090,17 +1090,17 @@ public class MetaTableAccessor {
* Adds a (single) hbase:meta row for the specified new region and its daughters. Note that this * Adds a (single) hbase:meta row for the specified new region and its daughters. Note that this
* does not add its daughter's as different rows, but adds information about the daughters * does not add its daughter's as different rows, but adds information about the daughters
* in the same row as the parent. Use * in the same row as the parent. Use
* {@link #splitRegion(HConnection, HRegionInfo, HRegionInfo, HRegionInfo, ServerName)} * {@link #splitRegion(Connection, HRegionInfo, HRegionInfo, HRegionInfo, ServerName)}
* if you want to do that. * if you want to do that.
* @param hConnection connection we're using * @param connection connection we're using
* @param regionInfo region information * @param regionInfo region information
* @param splitA first split daughter of the parent regionInfo * @param splitA first split daughter of the parent regionInfo
* @param splitB second split daughter of the parent regionInfo * @param splitB second split daughter of the parent regionInfo
* @throws IOException if problem connecting or updating meta * @throws IOException if problem connecting or updating meta
*/ */
public static void addRegionToMeta(HConnection hConnection, HRegionInfo regionInfo, public static void addRegionToMeta(Connection connection, HRegionInfo regionInfo,
HRegionInfo splitA, HRegionInfo splitB) throws IOException { HRegionInfo splitA, HRegionInfo splitB) throws IOException {
Table meta = getMetaHTable(hConnection); Table meta = getMetaHTable(connection);
try { try {
addRegionToMeta(meta, regionInfo, splitA, splitB); addRegionToMeta(meta, regionInfo, splitA, splitB);
} finally { } finally {
@ -1110,11 +1110,11 @@ public class MetaTableAccessor {
/** /**
* Adds a hbase:meta row for each of the specified new regions. * Adds a hbase:meta row for each of the specified new regions.
* @param hConnection connection we're using * @param connection connection we're using
* @param regionInfos region information list * @param regionInfos region information list
* @throws IOException if problem connecting or updating meta * @throws IOException if problem connecting or updating meta
*/ */
public static void addRegionsToMeta(HConnection hConnection, public static void addRegionsToMeta(Connection connection,
List<HRegionInfo> regionInfos) List<HRegionInfo> regionInfos)
throws IOException { throws IOException {
List<Put> puts = new ArrayList<Put>(); List<Put> puts = new ArrayList<Put>();
@ -1123,7 +1123,7 @@ public class MetaTableAccessor {
puts.add(makePutFromRegionInfo(regionInfo)); puts.add(makePutFromRegionInfo(regionInfo));
} }
} }
putsToMetaTable(hConnection, puts); putsToMetaTable(connection, puts);
LOG.info("Added " + puts.size()); LOG.info("Added " + puts.size());
} }
@ -1133,7 +1133,7 @@ public class MetaTableAccessor {
* @param sn the location of the region * @param sn the location of the region
* @param openSeqNum the latest sequence number obtained when the region was open * @param openSeqNum the latest sequence number obtained when the region was open
*/ */
public static void addDaughter(final HConnection hConnection, public static void addDaughter(final Connection connection,
final HRegionInfo regionInfo, final ServerName sn, final long openSeqNum) final HRegionInfo regionInfo, final ServerName sn, final long openSeqNum)
throws NotAllMetaRegionsOnlineException, IOException { throws NotAllMetaRegionsOnlineException, IOException {
Put put = new Put(regionInfo.getRegionName()); Put put = new Put(regionInfo.getRegionName());
@ -1141,7 +1141,7 @@ public class MetaTableAccessor {
if (sn != null) { if (sn != null) {
addLocation(put, sn, openSeqNum, regionInfo.getReplicaId()); addLocation(put, sn, openSeqNum, regionInfo.getReplicaId());
} }
putToMetaTable(hConnection, put); putToMetaTable(connection, put);
LOG.info("Added daughter " + regionInfo.getEncodedName() + LOG.info("Added daughter " + regionInfo.getEncodedName() +
(sn == null? ", serverName=null": ", serverName=" + sn.toString())); (sn == null? ", serverName=null": ", serverName=" + sn.toString()));
} }
@ -1150,16 +1150,16 @@ public class MetaTableAccessor {
* Merge the two regions into one in an atomic operation. Deletes the two * Merge the two regions into one in an atomic operation. Deletes the two
* merging regions in hbase:meta and adds the merged region with the information of * merging regions in hbase:meta and adds the merged region with the information of
* two merging regions. * two merging regions.
* @param hConnection connection we're using * @param connection connection we're using
* @param mergedRegion the merged region * @param mergedRegion the merged region
* @param regionA * @param regionA
* @param regionB * @param regionB
* @param sn the location of the region * @param sn the location of the region
* @throws IOException * @throws IOException
*/ */
public static void mergeRegions(final HConnection hConnection, HRegionInfo mergedRegion, public static void mergeRegions(final Connection connection, HRegionInfo mergedRegion,
HRegionInfo regionA, HRegionInfo regionB, ServerName sn) throws IOException { HRegionInfo regionA, HRegionInfo regionB, ServerName sn) throws IOException {
Table meta = getMetaHTable(hConnection); Table meta = getMetaHTable(connection);
try { try {
HRegionInfo copyOfMerged = new HRegionInfo(mergedRegion); HRegionInfo copyOfMerged = new HRegionInfo(mergedRegion);
@ -1190,16 +1190,16 @@ public class MetaTableAccessor {
* region with the information that it is split into two, and also adds * region with the information that it is split into two, and also adds
* the daughter regions. Does not add the location information to the daughter * the daughter regions. Does not add the location information to the daughter
* regions since they are not open yet. * regions since they are not open yet.
* @param hConnection connection we're using * @param connection connection we're using
* @param parent the parent region which is split * @param parent the parent region which is split
* @param splitA Split daughter region A * @param splitA Split daughter region A
* @param splitB Split daughter region A * @param splitB Split daughter region A
* @param sn the location of the region * @param sn the location of the region
*/ */
public static void splitRegion(final HConnection hConnection, public static void splitRegion(final Connection connection,
HRegionInfo parent, HRegionInfo splitA, HRegionInfo splitB, HRegionInfo parent, HRegionInfo splitA, HRegionInfo splitB,
ServerName sn) throws IOException { ServerName sn) throws IOException {
Table meta = getMetaHTable(hConnection); Table meta = getMetaHTable(connection);
try { try {
HRegionInfo copyOfParent = new HRegionInfo(parent); HRegionInfo copyOfParent = new HRegionInfo(parent);
copyOfParent.setOffline(true); copyOfParent.setOffline(true);
@ -1260,15 +1260,15 @@ public class MetaTableAccessor {
* Uses passed catalog tracker to get a connection to the server hosting * Uses passed catalog tracker to get a connection to the server hosting
* hbase:meta and makes edits to that region. * hbase:meta and makes edits to that region.
* *
* @param hConnection connection we're using * @param connection connection we're using
* @param regionInfo region to update location of * @param regionInfo region to update location of
* @param sn Server name * @param sn Server name
* @throws IOException * @throws IOException
*/ */
public static void updateRegionLocation(HConnection hConnection, public static void updateRegionLocation(Connection connection,
HRegionInfo regionInfo, ServerName sn, long updateSeqNum) HRegionInfo regionInfo, ServerName sn, long updateSeqNum)
throws IOException { throws IOException {
updateLocation(hConnection, regionInfo, sn, updateSeqNum); updateLocation(connection, regionInfo, sn, updateSeqNum);
} }
/** /**
@ -1277,62 +1277,62 @@ public class MetaTableAccessor {
* Connects to the specified server which should be hosting the specified * Connects to the specified server which should be hosting the specified
* catalog region name to perform the edit. * catalog region name to perform the edit.
* *
* @param hConnection connection we're using * @param connection connection we're using
* @param regionInfo region to update location of * @param regionInfo region to update location of
* @param sn Server name * @param sn Server name
* @param openSeqNum the latest sequence number obtained when the region was open * @param openSeqNum the latest sequence number obtained when the region was open
* @throws IOException In particular could throw {@link java.net.ConnectException} * @throws IOException In particular could throw {@link java.net.ConnectException}
* if the server is down on other end. * if the server is down on other end.
*/ */
private static void updateLocation(final HConnection hConnection, private static void updateLocation(final Connection connection,
HRegionInfo regionInfo, ServerName sn, long openSeqNum) HRegionInfo regionInfo, ServerName sn, long openSeqNum)
throws IOException { throws IOException {
// region replicas are kept in the primary region's row // region replicas are kept in the primary region's row
Put put = new Put(getMetaKeyForRegion(regionInfo)); Put put = new Put(getMetaKeyForRegion(regionInfo));
addLocation(put, sn, openSeqNum, regionInfo.getReplicaId()); addLocation(put, sn, openSeqNum, regionInfo.getReplicaId());
putToMetaTable(hConnection, put); putToMetaTable(connection, put);
LOG.info("Updated row " + regionInfo.getRegionNameAsString() + LOG.info("Updated row " + regionInfo.getRegionNameAsString() +
" with server=" + sn); " with server=" + sn);
} }
/** /**
* Deletes the specified region from META. * Deletes the specified region from META.
* @param hConnection connection we're using * @param connection connection we're using
* @param regionInfo region to be deleted from META * @param regionInfo region to be deleted from META
* @throws IOException * @throws IOException
*/ */
public static void deleteRegion(HConnection hConnection, public static void deleteRegion(Connection connection,
HRegionInfo regionInfo) HRegionInfo regionInfo)
throws IOException { throws IOException {
Delete delete = new Delete(regionInfo.getRegionName()); Delete delete = new Delete(regionInfo.getRegionName());
deleteFromMetaTable(hConnection, delete); deleteFromMetaTable(connection, delete);
LOG.info("Deleted " + regionInfo.getRegionNameAsString()); LOG.info("Deleted " + regionInfo.getRegionNameAsString());
} }
/** /**
* Deletes the specified regions from META. * Deletes the specified regions from META.
* @param hConnection connection we're using * @param connection connection we're using
* @param regionsInfo list of regions to be deleted from META * @param regionsInfo list of regions to be deleted from META
* @throws IOException * @throws IOException
*/ */
public static void deleteRegions(HConnection hConnection, public static void deleteRegions(Connection connection,
List<HRegionInfo> regionsInfo) throws IOException { List<HRegionInfo> regionsInfo) throws IOException {
List<Delete> deletes = new ArrayList<Delete>(regionsInfo.size()); List<Delete> deletes = new ArrayList<Delete>(regionsInfo.size());
for (HRegionInfo hri: regionsInfo) { for (HRegionInfo hri: regionsInfo) {
deletes.add(new Delete(hri.getRegionName())); deletes.add(new Delete(hri.getRegionName()));
} }
deleteFromMetaTable(hConnection, deletes); deleteFromMetaTable(connection, deletes);
LOG.info("Deleted " + regionsInfo); LOG.info("Deleted " + regionsInfo);
} }
/** /**
* Adds and Removes the specified regions from hbase:meta * Adds and Removes the specified regions from hbase:meta
* @param hConnection connection we're using * @param connection connection we're using
* @param regionsToRemove list of regions to be deleted from META * @param regionsToRemove list of regions to be deleted from META
* @param regionsToAdd list of regions to be added to META * @param regionsToAdd list of regions to be added to META
* @throws IOException * @throws IOException
*/ */
public static void mutateRegions(HConnection hConnection, public static void mutateRegions(Connection connection,
final List<HRegionInfo> regionsToRemove, final List<HRegionInfo> regionsToRemove,
final List<HRegionInfo> regionsToAdd) final List<HRegionInfo> regionsToAdd)
throws IOException { throws IOException {
@ -1347,7 +1347,7 @@ public class MetaTableAccessor {
mutation.add(makePutFromRegionInfo(hri)); mutation.add(makePutFromRegionInfo(hri));
} }
} }
mutateMetaTable(hConnection, mutation); mutateMetaTable(connection, mutation);
if (regionsToRemove != null && regionsToRemove.size() > 0) { if (regionsToRemove != null && regionsToRemove.size() > 0) {
LOG.debug("Deleted " + regionsToRemove); LOG.debug("Deleted " + regionsToRemove);
} }
@ -1358,34 +1358,34 @@ public class MetaTableAccessor {
/** /**
* Overwrites the specified regions from hbase:meta * Overwrites the specified regions from hbase:meta
* @param hConnection connection we're using * @param connection connection we're using
* @param regionInfos list of regions to be added to META * @param regionInfos list of regions to be added to META
* @throws IOException * @throws IOException
*/ */
public static void overwriteRegions(HConnection hConnection, public static void overwriteRegions(Connection connection,
List<HRegionInfo> regionInfos) throws IOException { List<HRegionInfo> regionInfos) throws IOException {
deleteRegions(hConnection, regionInfos); deleteRegions(connection, regionInfos);
// Why sleep? This is the easiest way to ensure that the previous deletes does not // Why sleep? This is the easiest way to ensure that the previous deletes does not
// eclipse the following puts, that might happen in the same ts from the server. // eclipse the following puts, that might happen in the same ts from the server.
// See HBASE-9906, and HBASE-9879. Once either HBASE-9879, HBASE-8770 is fixed, // See HBASE-9906, and HBASE-9879. Once either HBASE-9879, HBASE-8770 is fixed,
// or HBASE-9905 is fixed and meta uses seqIds, we do not need the sleep. // or HBASE-9905 is fixed and meta uses seqIds, we do not need the sleep.
Threads.sleep(20); Threads.sleep(20);
addRegionsToMeta(hConnection, regionInfos); addRegionsToMeta(connection, regionInfos);
LOG.info("Overwritten " + regionInfos); LOG.info("Overwritten " + regionInfos);
} }
/** /**
* Deletes merge qualifiers for the specified merged region. * Deletes merge qualifiers for the specified merged region.
* @param hConnection connection we're using * @param connection connection we're using
* @param mergedRegion * @param mergedRegion
* @throws IOException * @throws IOException
*/ */
public static void deleteMergeQualifiers(HConnection hConnection, public static void deleteMergeQualifiers(Connection connection,
final HRegionInfo mergedRegion) throws IOException { final HRegionInfo mergedRegion) throws IOException {
Delete delete = new Delete(mergedRegion.getRegionName()); Delete delete = new Delete(mergedRegion.getRegionName());
delete.deleteColumns(HConstants.CATALOG_FAMILY, HConstants.MERGEA_QUALIFIER); delete.deleteColumns(HConstants.CATALOG_FAMILY, HConstants.MERGEA_QUALIFIER);
delete.deleteColumns(HConstants.CATALOG_FAMILY, HConstants.MERGEB_QUALIFIER); delete.deleteColumns(HConstants.CATALOG_FAMILY, HConstants.MERGEB_QUALIFIER);
deleteFromMetaTable(hConnection, delete); deleteFromMetaTable(connection, delete);
LOG.info("Deleted references in merged region " LOG.info("Deleted references in merged region "
+ mergedRegion.getRegionNameAsString() + ", qualifier=" + mergedRegion.getRegionNameAsString() + ", qualifier="
+ Bytes.toStringBinary(HConstants.MERGEA_QUALIFIER) + " and qualifier=" + Bytes.toStringBinary(HConstants.MERGEA_QUALIFIER) + " and qualifier="

View File

@ -54,7 +54,7 @@ class ConnectionAdapter implements ClusterConnection {
private final ClusterConnection wrappedConnection; private final ClusterConnection wrappedConnection;
public ConnectionAdapter(HConnection c) { public ConnectionAdapter(Connection c) {
wrappedConnection = (ClusterConnection)c; wrappedConnection = (ClusterConnection)c;
} }

View File

@ -219,7 +219,7 @@ public class HBaseAdmin implements Admin {
* @deprecated Do not use this internal ctor. * @deprecated Do not use this internal ctor.
*/ */
@Deprecated @Deprecated
public HBaseAdmin(HConnection connection) public HBaseAdmin(Connection connection)
throws MasterNotRunningException, ZooKeeperConnectionException { throws MasterNotRunningException, ZooKeeperConnectionException {
this((ClusterConnection)connection); this((ClusterConnection)connection);
} }

View File

@ -268,14 +268,14 @@ public class HTable implements HTableInterface, RegionLocator {
* @deprecated Do not use, internal ctor. * @deprecated Do not use, internal ctor.
*/ */
@Deprecated @Deprecated
public HTable(final byte[] tableName, final HConnection connection, public HTable(final byte[] tableName, final Connection connection,
final ExecutorService pool) throws IOException { final ExecutorService pool) throws IOException {
this(TableName.valueOf(tableName), connection, pool); this(TableName.valueOf(tableName), connection, pool);
} }
/** @deprecated Do not use, internal ctor. */ /** @deprecated Do not use, internal ctor. */
@Deprecated @Deprecated
public HTable(TableName tableName, final HConnection connection, public HTable(TableName tableName, final Connection connection,
final ExecutorService pool) throws IOException { final ExecutorService pool) throws IOException {
this(tableName, (ClusterConnection)connection, pool); this(tableName, (ClusterConnection)connection, pool);
} }

View File

@ -30,7 +30,7 @@ interface Registry {
/** /**
* @param connection * @param connection
*/ */
void init(HConnection connection); void init(Connection connection);
/** /**
* @return Meta region location * @return Meta region location

View File

@ -41,7 +41,7 @@ class ZooKeeperRegistry implements Registry {
ConnectionManager.HConnectionImplementation hci; ConnectionManager.HConnectionImplementation hci;
@Override @Override
public void init(HConnection connection) { public void init(Connection connection) {
if (!(connection instanceof ConnectionManager.HConnectionImplementation)) { if (!(connection instanceof ConnectionManager.HConnectionImplementation)) {
throw new RuntimeException("This registry depends on HConnectionImplementation"); throw new RuntimeException("This registry depends on HConnectionImplementation");
} }

View File

@ -115,7 +115,7 @@ public class TestClientNoCluster extends Configured implements Tool {
final ServerName META_HOST = META_SERVERNAME; final ServerName META_HOST = META_SERVERNAME;
@Override @Override
public void init(HConnection connection) { public void init(Connection connection) {
} }
@Override @Override
@ -694,8 +694,8 @@ public class TestClientNoCluster extends Configured implements Tool {
* @param sharedConnection * @param sharedConnection
* @throws IOException * @throws IOException
*/ */
static void cycle(int id, final Configuration c, final HConnection sharedConnection) throws IOException { static void cycle(int id, final Configuration c, final Connection sharedConnection) throws IOException {
Table table = sharedConnection.getTable(BIG_USER_TABLE); Table table = sharedConnection.getTable(TableName.valueOf(BIG_USER_TABLE));
table.setAutoFlushTo(false); table.setAutoFlushTo(false);
long namespaceSpan = c.getLong("hbase.test.namespace.span", 1000000); long namespaceSpan = c.getLong("hbase.test.namespace.span", 1000000);
long startTime = System.currentTimeMillis(); long startTime = System.currentTimeMillis();
@ -772,7 +772,7 @@ public class TestClientNoCluster extends Configured implements Tool {
final ExecutorService pool = Executors.newCachedThreadPool(Threads.getNamedThreadFactory("p")); final ExecutorService pool = Executors.newCachedThreadPool(Threads.getNamedThreadFactory("p"));
// Executors.newFixedThreadPool(servers * 10, Threads.getNamedThreadFactory("p")); // Executors.newFixedThreadPool(servers * 10, Threads.getNamedThreadFactory("p"));
// Share a connection so I can keep counts in the 'server' on concurrency. // Share a connection so I can keep counts in the 'server' on concurrency.
final HConnection sharedConnection = HConnectionManager.createConnection(getConf()/*, pool*/); final Connection sharedConnection = ConnectionFactory.createConnection(getConf()/*, pool*/);
try { try {
Thread [] ts = new Thread[clients]; Thread [] ts = new Thread[clients];
for (int j = 0; j < ts.length; j++) { for (int j = 0; j < ts.length; j++) {

View File

@ -39,7 +39,7 @@ import org.apache.hadoop.hbase.ServerName;
import org.apache.hadoop.hbase.TableName; import org.apache.hadoop.hbase.TableName;
import org.apache.hadoop.hbase.MetaTableAccessor; import org.apache.hadoop.hbase.MetaTableAccessor;
import org.apache.hadoop.hbase.MetaTableAccessor.Visitor; import org.apache.hadoop.hbase.MetaTableAccessor.Visitor;
import org.apache.hadoop.hbase.client.HConnection; import org.apache.hadoop.hbase.client.Connection;
import org.apache.hadoop.hbase.client.Result; import org.apache.hadoop.hbase.client.Result;
import org.apache.hadoop.hbase.master.balancer.FavoredNodeAssignmentHelper; import org.apache.hadoop.hbase.master.balancer.FavoredNodeAssignmentHelper;
import org.apache.hadoop.hbase.master.balancer.FavoredNodesPlan; import org.apache.hadoop.hbase.master.balancer.FavoredNodesPlan;
@ -55,7 +55,7 @@ public class SnapshotOfRegionAssignmentFromMeta {
private static final Log LOG = LogFactory.getLog(SnapshotOfRegionAssignmentFromMeta.class private static final Log LOG = LogFactory.getLog(SnapshotOfRegionAssignmentFromMeta.class
.getName()); .getName());
private final HConnection hConnection; private final Connection connection;
/** the table name to region map */ /** the table name to region map */
private final Map<TableName, List<HRegionInfo>> tableToRegionMap; private final Map<TableName, List<HRegionInfo>> tableToRegionMap;
@ -72,13 +72,13 @@ public class SnapshotOfRegionAssignmentFromMeta {
private final Set<TableName> disabledTables; private final Set<TableName> disabledTables;
private final boolean excludeOfflinedSplitParents; private final boolean excludeOfflinedSplitParents;
public SnapshotOfRegionAssignmentFromMeta(HConnection hConnection) { public SnapshotOfRegionAssignmentFromMeta(Connection connection) {
this(hConnection, new HashSet<TableName>(), false); this(connection, new HashSet<TableName>(), false);
} }
public SnapshotOfRegionAssignmentFromMeta(HConnection hConnection, Set<TableName> disabledTables, public SnapshotOfRegionAssignmentFromMeta(Connection connection, Set<TableName> disabledTables,
boolean excludeOfflinedSplitParents) { boolean excludeOfflinedSplitParents) {
this.hConnection = hConnection; this.connection = connection;
tableToRegionMap = new HashMap<TableName, List<HRegionInfo>>(); tableToRegionMap = new HashMap<TableName, List<HRegionInfo>>();
regionToRegionServerMap = new HashMap<HRegionInfo, ServerName>(); regionToRegionServerMap = new HashMap<HRegionInfo, ServerName>();
regionServerToRegionMap = new HashMap<ServerName, List<HRegionInfo>>(); regionServerToRegionMap = new HashMap<ServerName, List<HRegionInfo>>();
@ -141,7 +141,7 @@ public class SnapshotOfRegionAssignmentFromMeta {
} }
}; };
// Scan hbase:meta to pick up user regions // Scan hbase:meta to pick up user regions
MetaTableAccessor.fullScan(hConnection, v); MetaTableAccessor.fullScan(connection, v);
//regionToRegionServerMap = regions; //regionToRegionServerMap = regions;
LOG.info("Finished to scan the hbase:meta for the current region assignment" + LOG.info("Finished to scan the hbase:meta for the current region assignment" +
"snapshot"); "snapshot");

View File

@ -38,7 +38,7 @@ import org.apache.hadoop.hbase.HConstants;
import org.apache.hadoop.hbase.HRegionInfo; import org.apache.hadoop.hbase.HRegionInfo;
import org.apache.hadoop.hbase.ServerName; import org.apache.hadoop.hbase.ServerName;
import org.apache.hadoop.hbase.MetaTableAccessor; import org.apache.hadoop.hbase.MetaTableAccessor;
import org.apache.hadoop.hbase.client.HConnection; import org.apache.hadoop.hbase.client.Connection;
import org.apache.hadoop.hbase.client.HTable; import org.apache.hadoop.hbase.client.HTable;
import org.apache.hadoop.hbase.client.Put; import org.apache.hadoop.hbase.client.Put;
import org.apache.hadoop.hbase.client.Table; import org.apache.hadoop.hbase.client.Table;
@ -87,12 +87,12 @@ public class FavoredNodeAssignmentHelper {
/** /**
* Update meta table with favored nodes info * Update meta table with favored nodes info
* @param regionToFavoredNodes map of HRegionInfo's to their favored nodes * @param regionToFavoredNodes map of HRegionInfo's to their favored nodes
* @param hConnection HConnection to be used * @param connection connection to be used
* @throws IOException * @throws IOException
*/ */
public static void updateMetaWithFavoredNodesInfo( public static void updateMetaWithFavoredNodesInfo(
Map<HRegionInfo, List<ServerName>> regionToFavoredNodes, Map<HRegionInfo, List<ServerName>> regionToFavoredNodes,
HConnection hConnection) throws IOException { Connection connection) throws IOException {
List<Put> puts = new ArrayList<Put>(); List<Put> puts = new ArrayList<Put>();
for (Map.Entry<HRegionInfo, List<ServerName>> entry : regionToFavoredNodes.entrySet()) { for (Map.Entry<HRegionInfo, List<ServerName>> entry : regionToFavoredNodes.entrySet()) {
Put put = makePutFromRegionInfo(entry.getKey(), entry.getValue()); Put put = makePutFromRegionInfo(entry.getKey(), entry.getValue());
@ -100,7 +100,7 @@ public class FavoredNodeAssignmentHelper {
puts.add(put); puts.add(put);
} }
} }
MetaTableAccessor.putsToMetaTable(hConnection, puts); MetaTableAccessor.putsToMetaTable(connection, puts);
LOG.info("Added " + puts.size() + " regions in META"); LOG.info("Added " + puts.size() + " regions in META");
} }

View File

@ -33,7 +33,7 @@ import org.apache.hadoop.hbase.HRegionInfo;
import org.apache.hadoop.hbase.HTableDescriptor; import org.apache.hadoop.hbase.HTableDescriptor;
import org.apache.hadoop.hbase.TableName; import org.apache.hadoop.hbase.TableName;
import org.apache.hadoop.hbase.MetaTableAccessor; import org.apache.hadoop.hbase.MetaTableAccessor;
import org.apache.hadoop.hbase.client.HConnection; import org.apache.hadoop.hbase.client.Connection;
import org.apache.hadoop.hbase.errorhandling.ForeignException; import org.apache.hadoop.hbase.errorhandling.ForeignException;
import org.apache.hadoop.hbase.errorhandling.ForeignExceptionDispatcher; import org.apache.hadoop.hbase.errorhandling.ForeignExceptionDispatcher;
import org.apache.hadoop.hbase.executor.EventType; import org.apache.hadoop.hbase.executor.EventType;
@ -109,7 +109,7 @@ public class RestoreSnapshotHandler extends TableEventHandler implements Snapsho
@Override @Override
protected void handleTableOperation(List<HRegionInfo> hris) throws IOException { protected void handleTableOperation(List<HRegionInfo> hris) throws IOException {
MasterFileSystem fileSystemManager = masterServices.getMasterFileSystem(); MasterFileSystem fileSystemManager = masterServices.getMasterFileSystem();
HConnection conn = masterServices.getShortCircuitConnection(); Connection conn = masterServices.getShortCircuitConnection();
FileSystem fs = fileSystemManager.getFileSystem(); FileSystem fs = fileSystemManager.getFileSystem();
Path rootDir = fileSystemManager.getRootDir(); Path rootDir = fileSystemManager.getRootDir();
TableName tableName = hTableDescriptor.getTableName(); TableName tableName = hTableDescriptor.getTableName();

View File

@ -42,9 +42,9 @@ import org.apache.hadoop.hbase.TableName;
import org.apache.hadoop.hbase.HBaseConfiguration; import org.apache.hadoop.hbase.HBaseConfiguration;
import org.apache.hadoop.hbase.HConstants; import org.apache.hadoop.hbase.HConstants;
import org.apache.hadoop.hbase.Stoppable; import org.apache.hadoop.hbase.Stoppable;
import org.apache.hadoop.hbase.client.Connection;
import org.apache.hadoop.hbase.client.ConnectionFactory;
import org.apache.hadoop.hbase.client.Delete; import org.apache.hadoop.hbase.client.Delete;
import org.apache.hadoop.hbase.client.HConnection;
import org.apache.hadoop.hbase.client.HConnectionManager;
import org.apache.hadoop.hbase.client.Mutation; import org.apache.hadoop.hbase.client.Mutation;
import org.apache.hadoop.hbase.client.Put; import org.apache.hadoop.hbase.client.Put;
import org.apache.hadoop.hbase.client.Row; import org.apache.hadoop.hbase.client.Row;
@ -71,7 +71,7 @@ public class ReplicationSink {
private static final Log LOG = LogFactory.getLog(ReplicationSink.class); private static final Log LOG = LogFactory.getLog(ReplicationSink.class);
private final Configuration conf; private final Configuration conf;
private final HConnection sharedHtableCon; private final Connection sharedHtableCon;
private final MetricsSink metrics; private final MetricsSink metrics;
private final AtomicLong totalReplicatedEdits = new AtomicLong(); private final AtomicLong totalReplicatedEdits = new AtomicLong();
@ -87,7 +87,7 @@ public class ReplicationSink {
this.conf = HBaseConfiguration.create(conf); this.conf = HBaseConfiguration.create(conf);
decorateConf(); decorateConf();
this.metrics = new MetricsSink(); this.metrics = new MetricsSink();
this.sharedHtableCon = HConnectionManager.createConnection(this.conf); this.sharedHtableCon = ConnectionFactory.createConnection(this.conf);
} }
/** /**

View File

@ -45,7 +45,7 @@ import org.apache.hadoop.hbase.HTableDescriptor;
import org.apache.hadoop.hbase.TableName; import org.apache.hadoop.hbase.TableName;
import org.apache.hadoop.hbase.backup.HFileArchiver; import org.apache.hadoop.hbase.backup.HFileArchiver;
import org.apache.hadoop.hbase.MetaTableAccessor; import org.apache.hadoop.hbase.MetaTableAccessor;
import org.apache.hadoop.hbase.client.HConnection; import org.apache.hadoop.hbase.client.Connection;
import org.apache.hadoop.hbase.errorhandling.ForeignExceptionDispatcher; import org.apache.hadoop.hbase.errorhandling.ForeignExceptionDispatcher;
import org.apache.hadoop.hbase.io.HFileLink; import org.apache.hadoop.hbase.io.HFileLink;
import org.apache.hadoop.hbase.io.Reference; import org.apache.hadoop.hbase.io.Reference;
@ -58,7 +58,6 @@ import org.apache.hadoop.hbase.regionserver.HRegionFileSystem;
import org.apache.hadoop.hbase.regionserver.StoreFileInfo; import org.apache.hadoop.hbase.regionserver.StoreFileInfo;
import org.apache.hadoop.hbase.util.Bytes; import org.apache.hadoop.hbase.util.Bytes;
import org.apache.hadoop.hbase.util.FSUtils; import org.apache.hadoop.hbase.util.FSUtils;
import org.apache.hadoop.hbase.util.Bytes;
import org.apache.hadoop.hbase.util.ModifyRegionUtils; import org.apache.hadoop.hbase.util.ModifyRegionUtils;
import org.apache.hadoop.hbase.util.Pair; import org.apache.hadoop.hbase.util.Pair;
import org.apache.hadoop.io.IOUtils; import org.apache.hadoop.io.IOUtils;
@ -313,13 +312,13 @@ public class RestoreSnapshotHelper {
regionsToRestore.add(hri); regionsToRestore.add(hri);
} }
public void updateMetaParentRegions(HConnection hConnection, public void updateMetaParentRegions(Connection connection,
final List<HRegionInfo> regionInfos) throws IOException { final List<HRegionInfo> regionInfos) throws IOException {
if (regionInfos == null || parentsMap.isEmpty()) return; if (regionInfos == null || parentsMap.isEmpty()) return;
// Extract region names and offlined regions // Extract region names and offlined regions
Map<String, HRegionInfo> regionsByName = new HashMap<String, HRegionInfo>(regionInfos.size()); Map<String, HRegionInfo> regionsByName = new HashMap<String, HRegionInfo>(regionInfos.size());
List<HRegionInfo> parentRegions = new LinkedList(); List<HRegionInfo> parentRegions = new LinkedList<>();
for (HRegionInfo regionInfo: regionInfos) { for (HRegionInfo regionInfo: regionInfos) {
if (regionInfo.isSplitParent()) { if (regionInfo.isSplitParent()) {
parentRegions.add(regionInfo); parentRegions.add(regionInfo);
@ -344,7 +343,7 @@ public class RestoreSnapshotHelper {
} }
LOG.debug("Update splits parent " + regionInfo.getEncodedName() + " -> " + daughters); LOG.debug("Update splits parent " + regionInfo.getEncodedName() + " -> " + daughters);
MetaTableAccessor.addRegionToMeta(hConnection, regionInfo, MetaTableAccessor.addRegionToMeta(connection, regionInfo,
regionsByName.get(daughters.getFirst()), regionsByName.get(daughters.getFirst()),
regionsByName.get(daughters.getSecond())); regionsByName.get(daughters.getSecond()));
} }

View File

@ -34,6 +34,7 @@ import org.apache.hadoop.hbase.classification.InterfaceAudience;
import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.hbase.HConstants; import org.apache.hadoop.hbase.HConstants;
import org.apache.hadoop.hbase.TableName; import org.apache.hadoop.hbase.TableName;
import org.apache.hadoop.hbase.client.Connection;
import org.apache.hadoop.hbase.client.HConnection; import org.apache.hadoop.hbase.client.HConnection;
import org.apache.hadoop.hbase.client.HConnectionManager; import org.apache.hadoop.hbase.client.HConnectionManager;
import org.apache.hadoop.hbase.client.Row; import org.apache.hadoop.hbase.client.Row;
@ -74,7 +75,7 @@ public class MultiHConnection {
if (hConnections != null) { if (hConnections != null) {
synchronized (hConnections) { synchronized (hConnections) {
if (hConnections != null) { if (hConnections != null) {
for (HConnection conn : hConnections) { for (Connection conn : hConnections) {
if (conn != null) { if (conn != null) {
try { try {
conn.close(); conn.close();

View File

@ -56,12 +56,13 @@ import org.apache.hadoop.fs.FileSystem;
import org.apache.hadoop.fs.Path; import org.apache.hadoop.fs.Path;
import org.apache.hadoop.hbase.Waiter.Predicate; import org.apache.hadoop.hbase.Waiter.Predicate;
import org.apache.hadoop.hbase.client.Admin; 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.Delete; import org.apache.hadoop.hbase.client.Delete;
import org.apache.hadoop.hbase.client.Durability; import org.apache.hadoop.hbase.client.Durability;
import org.apache.hadoop.hbase.client.Get; import org.apache.hadoop.hbase.client.Get;
import org.apache.hadoop.hbase.client.HBaseAdmin; import org.apache.hadoop.hbase.client.HBaseAdmin;
import org.apache.hadoop.hbase.client.HConnection; import org.apache.hadoop.hbase.client.HConnection;
import org.apache.hadoop.hbase.client.HConnectionManager;
import org.apache.hadoop.hbase.client.HTable; import org.apache.hadoop.hbase.client.HTable;
import org.apache.hadoop.hbase.client.Put; import org.apache.hadoop.hbase.client.Put;
import org.apache.hadoop.hbase.client.RegionLocator; import org.apache.hadoop.hbase.client.RegionLocator;
@ -3374,7 +3375,7 @@ public class HBaseTestingUtility extends HBaseCommonTestingUtility {
} }
int totalNumberOfRegions = 0; int totalNumberOfRegions = 0;
HConnection unmanagedConnection = HConnectionManager.createConnection(conf); Connection unmanagedConnection = ConnectionFactory.createConnection(conf);
Admin admin = unmanagedConnection.getAdmin(); Admin admin = unmanagedConnection.getAdmin();
try { try {

View File

@ -48,12 +48,12 @@ import org.apache.hadoop.conf.Configured;
import org.apache.hadoop.fs.FileSystem; import org.apache.hadoop.fs.FileSystem;
import org.apache.hadoop.fs.Path; import org.apache.hadoop.fs.Path;
import org.apache.hadoop.hbase.client.Admin; 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.Consistency; import org.apache.hadoop.hbase.client.Consistency;
import org.apache.hadoop.hbase.client.Durability; import org.apache.hadoop.hbase.client.Durability;
import org.apache.hadoop.hbase.client.Get; import org.apache.hadoop.hbase.client.Get;
import org.apache.hadoop.hbase.client.HBaseAdmin; import org.apache.hadoop.hbase.client.HBaseAdmin;
import org.apache.hadoop.hbase.client.HConnection;
import org.apache.hadoop.hbase.client.HConnectionManager;
import org.apache.hadoop.hbase.client.HTable; import org.apache.hadoop.hbase.client.HTable;
import org.apache.hadoop.hbase.client.Put; import org.apache.hadoop.hbase.client.Put;
import org.apache.hadoop.hbase.client.Result; import org.apache.hadoop.hbase.client.Result;
@ -255,7 +255,7 @@ public class PerformanceEvaluation extends Configured implements Tool {
ObjectMapper mapper = new ObjectMapper(); ObjectMapper mapper = new ObjectMapper();
TestOptions opts = mapper.readValue(value.toString(), TestOptions.class); TestOptions opts = mapper.readValue(value.toString(), TestOptions.class);
Configuration conf = HBaseConfiguration.create(context.getConfiguration()); Configuration conf = HBaseConfiguration.create(context.getConfiguration());
final HConnection con = HConnectionManager.createConnection(conf); final Connection con = ConnectionFactory.createConnection(conf);
// Evaluation task // Evaluation task
long elapsedTime = runOneClient(this.cmd, conf, con, opts, status); long elapsedTime = runOneClient(this.cmd, conf, con, opts, status);
@ -379,7 +379,7 @@ public class PerformanceEvaluation extends Configured implements Tool {
long[] timings = new long[opts.numClientThreads]; long[] timings = new long[opts.numClientThreads];
ExecutorService pool = Executors.newFixedThreadPool(opts.numClientThreads, ExecutorService pool = Executors.newFixedThreadPool(opts.numClientThreads,
new ThreadFactoryBuilder().setNameFormat("TestClient-%s").build()); new ThreadFactoryBuilder().setNameFormat("TestClient-%s").build());
final HConnection con = HConnectionManager.createConnection(conf); final Connection con = ConnectionFactory.createConnection(conf);
for (int i = 0; i < threads.length; i++) { for (int i = 0; i < threads.length; i++) {
final int index = i; final int index = i;
threads[i] = pool.submit(new Callable<Long>() { threads[i] = pool.submit(new Callable<Long>() {
@ -922,7 +922,7 @@ public class PerformanceEvaluation extends Configured implements Tool {
private final Status status; private final Status status;
private final Sampler<?> traceSampler; private final Sampler<?> traceSampler;
private final SpanReceiverHost receiverHost; private final SpanReceiverHost receiverHost;
protected HConnection connection; protected Connection connection;
protected Table table; protected Table table;
private String testName; private String testName;
@ -934,7 +934,7 @@ public class PerformanceEvaluation extends Configured implements Tool {
* Note that all subclasses of this class must provide a public constructor * Note that all subclasses of this class must provide a public constructor
* that has the exact same list of arguments. * that has the exact same list of arguments.
*/ */
Test(final HConnection con, final TestOptions options, final Status status) { Test(final Connection con, final TestOptions options, final Status status) {
this.connection = con; this.connection = con;
this.conf = con == null? null: this.connection.getConfiguration(); this.conf = con == null? null: this.connection.getConfiguration();
this.receiverHost = this.conf == null? null: SpanReceiverHost.getInstance(conf); this.receiverHost = this.conf == null? null: SpanReceiverHost.getInstance(conf);
@ -995,7 +995,7 @@ public class PerformanceEvaluation extends Configured implements Tool {
void testSetup() throws IOException { void testSetup() throws IOException {
if (!opts.oneCon) { if (!opts.oneCon) {
this.connection = HConnectionManager.createConnection(conf); this.connection = ConnectionFactory.createConnection(conf);
} }
this.table = new HTable(TableName.valueOf(opts.tableName), connection); this.table = new HTable(TableName.valueOf(opts.tableName), connection);
this.table.setAutoFlushTo(opts.autoFlush); this.table.setAutoFlushTo(opts.autoFlush);
@ -1135,7 +1135,7 @@ public class PerformanceEvaluation extends Configured implements Tool {
} }
static class RandomSeekScanTest extends Test { static class RandomSeekScanTest extends Test {
RandomSeekScanTest(HConnection con, TestOptions options, Status status) { RandomSeekScanTest(Connection con, TestOptions options, Status status) {
super(con, options, status); super(con, options, status);
} }
@ -1165,7 +1165,7 @@ public class PerformanceEvaluation extends Configured implements Tool {
} }
static abstract class RandomScanWithRangeTest extends Test { static abstract class RandomScanWithRangeTest extends Test {
RandomScanWithRangeTest(HConnection con, TestOptions options, Status status) { RandomScanWithRangeTest(Connection con, TestOptions options, Status status) {
super(con, options, status); super(con, options, status);
} }
@ -1209,7 +1209,7 @@ public class PerformanceEvaluation extends Configured implements Tool {
} }
static class RandomScanWithRange10Test extends RandomScanWithRangeTest { static class RandomScanWithRange10Test extends RandomScanWithRangeTest {
RandomScanWithRange10Test(HConnection con, TestOptions options, Status status) { RandomScanWithRange10Test(Connection con, TestOptions options, Status status) {
super(con, options, status); super(con, options, status);
} }
@ -1220,7 +1220,7 @@ public class PerformanceEvaluation extends Configured implements Tool {
} }
static class RandomScanWithRange100Test extends RandomScanWithRangeTest { static class RandomScanWithRange100Test extends RandomScanWithRangeTest {
RandomScanWithRange100Test(HConnection con, TestOptions options, Status status) { RandomScanWithRange100Test(Connection con, TestOptions options, Status status) {
super(con, options, status); super(con, options, status);
} }
@ -1231,7 +1231,7 @@ public class PerformanceEvaluation extends Configured implements Tool {
} }
static class RandomScanWithRange1000Test extends RandomScanWithRangeTest { static class RandomScanWithRange1000Test extends RandomScanWithRangeTest {
RandomScanWithRange1000Test(HConnection con, TestOptions options, Status status) { RandomScanWithRange1000Test(Connection con, TestOptions options, Status status) {
super(con, options, status); super(con, options, status);
} }
@ -1242,7 +1242,7 @@ public class PerformanceEvaluation extends Configured implements Tool {
} }
static class RandomScanWithRange10000Test extends RandomScanWithRangeTest { static class RandomScanWithRange10000Test extends RandomScanWithRangeTest {
RandomScanWithRange10000Test(HConnection con, TestOptions options, Status status) { RandomScanWithRange10000Test(Connection con, TestOptions options, Status status) {
super(con, options, status); super(con, options, status);
} }
@ -1257,7 +1257,7 @@ public class PerformanceEvaluation extends Configured implements Tool {
private ArrayList<Get> gets; private ArrayList<Get> gets;
private Random rd = new Random(); private Random rd = new Random();
RandomReadTest(HConnection con, TestOptions options, Status status) { RandomReadTest(Connection con, TestOptions options, Status status) {
super(con, options, status); super(con, options, status);
consistency = options.replicas == DEFAULT_OPTS.replicas ? null : Consistency.TIMELINE; consistency = options.replicas == DEFAULT_OPTS.replicas ? null : Consistency.TIMELINE;
if (opts.multiGet > 0) { if (opts.multiGet > 0) {
@ -1307,7 +1307,7 @@ public class PerformanceEvaluation extends Configured implements Tool {
} }
static class RandomWriteTest extends Test { static class RandomWriteTest extends Test {
RandomWriteTest(HConnection con, TestOptions options, Status status) { RandomWriteTest(Connection con, TestOptions options, Status status) {
super(con, options, status); super(con, options, status);
} }
@ -1339,7 +1339,7 @@ public class PerformanceEvaluation extends Configured implements Tool {
static class ScanTest extends Test { static class ScanTest extends Test {
private ResultScanner testScanner; private ResultScanner testScanner;
ScanTest(HConnection con, TestOptions options, Status status) { ScanTest(Connection con, TestOptions options, Status status) {
super(con, options, status); super(con, options, status);
} }
@ -1370,7 +1370,7 @@ public class PerformanceEvaluation extends Configured implements Tool {
} }
static class SequentialReadTest extends Test { static class SequentialReadTest extends Test {
SequentialReadTest(HConnection con, TestOptions options, Status status) { SequentialReadTest(Connection con, TestOptions options, Status status) {
super(con, options, status); super(con, options, status);
} }
@ -1386,7 +1386,7 @@ public class PerformanceEvaluation extends Configured implements Tool {
} }
static class SequentialWriteTest extends Test { static class SequentialWriteTest extends Test {
SequentialWriteTest(HConnection con, TestOptions options, Status status) { SequentialWriteTest(Connection con, TestOptions options, Status status) {
super(con, options, status); super(con, options, status);
} }
@ -1418,7 +1418,7 @@ public class PerformanceEvaluation extends Configured implements Tool {
static class FilteredScanTest extends Test { static class FilteredScanTest extends Test {
protected static final Log LOG = LogFactory.getLog(FilteredScanTest.class.getName()); protected static final Log LOG = LogFactory.getLog(FilteredScanTest.class.getName());
FilteredScanTest(HConnection con, TestOptions options, Status status) { FilteredScanTest(Connection con, TestOptions options, Status status) {
super(con, options, status); super(con, options, status);
} }
@ -1528,7 +1528,7 @@ public class PerformanceEvaluation extends Configured implements Tool {
return format(random.nextInt(Integer.MAX_VALUE) % totalRows); return format(random.nextInt(Integer.MAX_VALUE) % totalRows);
} }
static long runOneClient(final Class<? extends Test> cmd, Configuration conf, HConnection con, static long runOneClient(final Class<? extends Test> cmd, Configuration conf, Connection con,
TestOptions opts, final Status status) TestOptions opts, final Status status)
throws IOException, InterruptedException { throws IOException, InterruptedException {
status.setStatus("Start " + cmd + " at offset " + opts.startRow + " for " + status.setStatus("Start " + cmd + " at offset " + opts.startRow + " for " +
@ -1538,7 +1538,7 @@ public class PerformanceEvaluation extends Configured implements Tool {
final Test t; final Test t;
try { try {
Constructor<? extends Test> constructor = Constructor<? extends Test> constructor =
cmd.getDeclaredConstructor(HConnection.class, TestOptions.class, Status.class); cmd.getDeclaredConstructor(Connection.class, TestOptions.class, Status.class);
t = constructor.newInstance(con, opts, status); t = constructor.newInstance(con, opts, status);
} catch (NoSuchMethodException e) { } catch (NoSuchMethodException e) {
throw new IllegalArgumentException("Invalid command class: " + throw new IllegalArgumentException("Invalid command class: " +

View File

@ -31,9 +31,9 @@ import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory; import org.apache.commons.logging.LogFactory;
import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.hbase.client.Admin; import org.apache.hadoop.hbase.client.Admin;
import org.apache.hadoop.hbase.client.Get; import org.apache.hadoop.hbase.client.Connection;
import org.apache.hadoop.hbase.client.HConnection;
import org.apache.hadoop.hbase.client.HConnectionManager; import org.apache.hadoop.hbase.client.HConnectionManager;
import org.apache.hadoop.hbase.client.Get;
import org.apache.hadoop.hbase.client.HTable; import org.apache.hadoop.hbase.client.HTable;
import org.apache.hadoop.hbase.client.Result; import org.apache.hadoop.hbase.client.Result;
import org.apache.hadoop.hbase.client.Table; import org.apache.hadoop.hbase.client.Table;
@ -56,7 +56,7 @@ import org.junit.experimental.categories.Category;
public class TestMetaTableAccessor { public class TestMetaTableAccessor {
private static final Log LOG = LogFactory.getLog(TestMetaTableAccessor.class); private static final Log LOG = LogFactory.getLog(TestMetaTableAccessor.class);
private static final HBaseTestingUtility UTIL = new HBaseTestingUtility(); private static final HBaseTestingUtility UTIL = new HBaseTestingUtility();
private static HConnection hConnection; private static Connection connection;
@BeforeClass public static void beforeClass() throws Exception { @BeforeClass public static void beforeClass() throws Exception {
UTIL.startMiniCluster(3); UTIL.startMiniCluster(3);
@ -66,7 +66,7 @@ public class TestMetaTableAccessor {
// responsive. 1 second is default as is ten retries. // responsive. 1 second is default as is ten retries.
c.setLong("hbase.client.pause", 1000); c.setLong("hbase.client.pause", 1000);
c.setInt(HConstants.HBASE_CLIENT_RETRIES_NUMBER, 10); c.setInt(HConstants.HBASE_CLIENT_RETRIES_NUMBER, 10);
hConnection = HConnectionManager.getConnection(c); connection = HConnectionManager.getConnection(c);
} }
@AfterClass public static void afterClass() throws Exception { @AfterClass public static void afterClass() throws Exception {
@ -74,7 +74,7 @@ public class TestMetaTableAccessor {
} }
/** /**
* Does {@link MetaTableAccessor#getRegion(HConnection, byte[])} and a write * Does {@link MetaTableAccessor#getRegion(Connection, byte[])} and a write
* against hbase:meta while its hosted server is restarted to prove our retrying * against hbase:meta while its hosted server is restarted to prove our retrying
* works. * works.
* @throws IOException * @throws IOException
@ -89,18 +89,18 @@ public class TestMetaTableAccessor {
int regionCount = UTIL.createMultiRegions(t, HConstants.CATALOG_FAMILY); int regionCount = UTIL.createMultiRegions(t, HConstants.CATALOG_FAMILY);
// Test it works getting a region from just made user table. // Test it works getting a region from just made user table.
final List<HRegionInfo> regions = final List<HRegionInfo> regions =
testGettingTableRegions(hConnection, name, regionCount); testGettingTableRegions(connection, name, regionCount);
MetaTask reader = new MetaTask(hConnection, "reader") { MetaTask reader = new MetaTask(connection, "reader") {
@Override @Override
void metaTask() throws Throwable { void metaTask() throws Throwable {
testGetRegion(hConnection, regions.get(0)); testGetRegion(connection, regions.get(0));
LOG.info("Read " + regions.get(0).getEncodedName()); LOG.info("Read " + regions.get(0).getEncodedName());
} }
}; };
MetaTask writer = new MetaTask(hConnection, "writer") { MetaTask writer = new MetaTask(connection, "writer") {
@Override @Override
void metaTask() throws Throwable { void metaTask() throws Throwable {
MetaTableAccessor.addRegionToMeta(hConnection, regions.get(0)); MetaTableAccessor.addRegionToMeta(connection, regions.get(0));
LOG.info("Wrote " + regions.get(0).getEncodedName()); LOG.info("Wrote " + regions.get(0).getEncodedName());
} }
}; };
@ -157,11 +157,11 @@ public class TestMetaTableAccessor {
boolean stop = false; boolean stop = false;
int count = 0; int count = 0;
Throwable t = null; Throwable t = null;
final HConnection hConnection; final Connection connection;
MetaTask(final HConnection hConnection, final String name) { MetaTask(final Connection connection, final String name) {
super(name); super(name);
this.hConnection = hConnection; this.connection = connection;
} }
@Override @Override
@ -211,14 +211,14 @@ public class TestMetaTableAccessor {
@Test public void testTableExists() throws IOException { @Test public void testTableExists() throws IOException {
final TableName name = final TableName name =
TableName.valueOf("testTableExists"); TableName.valueOf("testTableExists");
assertFalse(MetaTableAccessor.tableExists(hConnection, name)); assertFalse(MetaTableAccessor.tableExists(connection, name));
UTIL.createTable(name, HConstants.CATALOG_FAMILY); UTIL.createTable(name, HConstants.CATALOG_FAMILY);
assertTrue(MetaTableAccessor.tableExists(hConnection, name)); assertTrue(MetaTableAccessor.tableExists(connection, name));
Admin admin = UTIL.getHBaseAdmin(); Admin admin = UTIL.getHBaseAdmin();
admin.disableTable(name); admin.disableTable(name);
admin.deleteTable(name); admin.deleteTable(name);
assertFalse(MetaTableAccessor.tableExists(hConnection, name)); assertFalse(MetaTableAccessor.tableExists(connection, name));
assertTrue(MetaTableAccessor.tableExists(hConnection, assertTrue(MetaTableAccessor.tableExists(connection,
TableName.META_TABLE_NAME)); TableName.META_TABLE_NAME));
} }
@ -227,7 +227,7 @@ public class TestMetaTableAccessor {
LOG.info("Started " + name); LOG.info("Started " + name);
// Test get on non-existent region. // Test get on non-existent region.
Pair<HRegionInfo, ServerName> pair = Pair<HRegionInfo, ServerName> pair =
MetaTableAccessor.getRegion(hConnection, Bytes.toBytes("nonexistent-region")); MetaTableAccessor.getRegion(connection, Bytes.toBytes("nonexistent-region"));
assertNull(pair); assertNull(pair);
LOG.info("Finished " + name); LOG.info("Finished " + name);
} }
@ -252,27 +252,27 @@ public class TestMetaTableAccessor {
// Now make sure we only get the regions from 1 of the tables at a time // Now make sure we only get the regions from 1 of the tables at a time
assertEquals(1, MetaTableAccessor.getTableRegions(hConnection, name).size()); assertEquals(1, MetaTableAccessor.getTableRegions(connection, name).size());
assertEquals(1, MetaTableAccessor.getTableRegions(hConnection, greaterName).size()); assertEquals(1, MetaTableAccessor.getTableRegions(connection, greaterName).size());
} }
private static List<HRegionInfo> testGettingTableRegions(final HConnection hConnection, private static List<HRegionInfo> testGettingTableRegions(final Connection connection,
final TableName name, final int regionCount) final TableName name, final int regionCount)
throws IOException, InterruptedException { throws IOException, InterruptedException {
List<HRegionInfo> regions = MetaTableAccessor.getTableRegions(hConnection, name); List<HRegionInfo> regions = MetaTableAccessor.getTableRegions(connection, name);
assertEquals(regionCount, regions.size()); assertEquals(regionCount, regions.size());
Pair<HRegionInfo, ServerName> pair = Pair<HRegionInfo, ServerName> pair =
MetaTableAccessor.getRegion(hConnection, regions.get(0).getRegionName()); MetaTableAccessor.getRegion(connection, regions.get(0).getRegionName());
assertEquals(regions.get(0).getEncodedName(), assertEquals(regions.get(0).getEncodedName(),
pair.getFirst().getEncodedName()); pair.getFirst().getEncodedName());
return regions; return regions;
} }
private static void testGetRegion(final HConnection hConnection, private static void testGetRegion(final Connection connection,
final HRegionInfo region) final HRegionInfo region)
throws IOException, InterruptedException { throws IOException, InterruptedException {
Pair<HRegionInfo, ServerName> pair = Pair<HRegionInfo, ServerName> pair =
MetaTableAccessor.getRegion(hConnection, region.getRegionName()); MetaTableAccessor.getRegion(connection, region.getRegionName());
assertEquals(region.getEncodedName(), assertEquals(region.getEncodedName(),
pair.getFirst().getEncodedName()); pair.getFirst().getEncodedName());
} }
@ -333,22 +333,22 @@ public class TestMetaTableAccessor {
long seqNum100 = random.nextLong(); long seqNum100 = random.nextLong();
Table meta = MetaTableAccessor.getMetaHTable(hConnection); Table meta = MetaTableAccessor.getMetaHTable(connection);
try { try {
MetaTableAccessor.updateRegionLocation(hConnection, primary, serverName0, seqNum0); MetaTableAccessor.updateRegionLocation(connection, primary, serverName0, seqNum0);
// assert that the server, startcode and seqNum columns are there for the primary region // assert that the server, startcode and seqNum columns are there for the primary region
assertMetaLocation(meta, primary.getRegionName(), serverName0, seqNum0, 0, true); assertMetaLocation(meta, primary.getRegionName(), serverName0, seqNum0, 0, true);
// add replica = 1 // add replica = 1
MetaTableAccessor.updateRegionLocation(hConnection, replica1, serverName1, seqNum1); MetaTableAccessor.updateRegionLocation(connection, replica1, serverName1, seqNum1);
// check whether the primary is still there // check whether the primary is still there
assertMetaLocation(meta, primary.getRegionName(), serverName0, seqNum0, 0, true); assertMetaLocation(meta, primary.getRegionName(), serverName0, seqNum0, 0, true);
// now check for replica 1 // now check for replica 1
assertMetaLocation(meta, primary.getRegionName(), serverName1, seqNum1, 1, true); assertMetaLocation(meta, primary.getRegionName(), serverName1, seqNum1, 1, true);
// add replica = 1 // add replica = 1
MetaTableAccessor.updateRegionLocation(hConnection, replica100, serverName100, seqNum100); MetaTableAccessor.updateRegionLocation(connection, replica100, serverName100, seqNum100);
// check whether the primary is still there // check whether the primary is still there
assertMetaLocation(meta, primary.getRegionName(), serverName0, seqNum0, 0, true); assertMetaLocation(meta, primary.getRegionName(), serverName0, seqNum0, 0, true);
// check whether the replica 1 is still there // check whether the replica 1 is still there

View File

@ -78,7 +78,7 @@ public class TestClientTimeouts {
*/ */
@Test @Test
public void testAdminTimeout() throws Exception { public void testAdminTimeout() throws Exception {
HConnection lastConnection = null; Connection lastConnection = null;
boolean lastFailed = false; boolean lastFailed = false;
int initialInvocations = RandomTimeoutBlockingRpcChannel.invokations.get(); int initialInvocations = RandomTimeoutBlockingRpcChannel.invokations.get();
RpcClient rpcClient = newRandomTimeoutRpcClient(); RpcClient rpcClient = newRandomTimeoutRpcClient();
@ -91,7 +91,7 @@ public class TestClientTimeouts {
HBaseAdmin admin = null; HBaseAdmin admin = null;
try { try {
admin = new HBaseAdmin(conf); admin = new HBaseAdmin(conf);
HConnection connection = admin.getConnection(); Connection connection = admin.getConnection();
assertFalse(connection == lastConnection); assertFalse(connection == lastConnection);
lastConnection = connection; lastConnection = connection;
// Override the connection's rpc client for timeout testing // Override the connection's rpc client for timeout testing

View File

@ -4127,7 +4127,7 @@ public class TestFromClientSide {
public void testUnmanagedHConnectionReconnect() throws Exception { public void testUnmanagedHConnectionReconnect() throws Exception {
final byte[] tableName = Bytes.toBytes("testUnmanagedHConnectionReconnect"); final byte[] tableName = Bytes.toBytes("testUnmanagedHConnectionReconnect");
HTable t = createUnmangedHConnectionHTable(tableName); HTable t = createUnmangedHConnectionHTable(tableName);
HConnection conn = t.getConnection(); Connection conn = t.getConnection();
HBaseAdmin ha = new HBaseAdmin(conn); HBaseAdmin ha = new HBaseAdmin(conn);
assertTrue(ha.tableExists(tableName)); assertTrue(ha.tableExists(tableName));
assertTrue(t.get(new Get(ROW)).isEmpty()); assertTrue(t.get(new Get(ROW)).isEmpty());

View File

@ -212,7 +212,7 @@ public class TestHCM {
*/ */
@Test @Test
public void testAdminFactory() throws IOException { public void testAdminFactory() throws IOException {
HConnection con1 = HConnectionManager.createConnection(TEST_UTIL.getConfiguration()); Connection con1 = ConnectionFactory.createConnection(TEST_UTIL.getConfiguration());
Admin admin = con1.getAdmin(); Admin admin = con1.getAdmin();
assertTrue(admin.getConnection() == con1); assertTrue(admin.getConnection() == con1);
assertTrue(admin.getConfiguration() == TEST_UTIL.getConfiguration()); assertTrue(admin.getConfiguration() == TEST_UTIL.getConfiguration());
@ -779,16 +779,16 @@ public class TestHCM {
@Test @Test
public void testConnectionManagement() throws Exception{ public void testConnectionManagement() throws Exception{
Table table0 = TEST_UTIL.createTable(TABLE_NAME1, FAM_NAM); Table table0 = TEST_UTIL.createTable(TABLE_NAME1, FAM_NAM);
HConnection conn = HConnectionManager.createConnection(TEST_UTIL.getConfiguration()); Connection conn = ConnectionFactory.createConnection(TEST_UTIL.getConfiguration());
Table table = conn.getTable(TABLE_NAME1.getName()); HTable table = (HTable) conn.getTable(TABLE_NAME1);
table.close(); table.close();
assertFalse(conn.isClosed()); assertFalse(conn.isClosed());
assertFalse(((HTable)table).getPool().isShutdown()); assertFalse(table.getPool().isShutdown());
table = conn.getTable(TABLE_NAME1.getName()); table = (HTable) conn.getTable(TABLE_NAME1);
table.close(); table.close();
assertFalse(((HTable)table).getPool().isShutdown()); assertFalse(table.getPool().isShutdown());
conn.close(); conn.close();
assertTrue(((HTable)table).getPool().isShutdown()); assertTrue(table.getPool().isShutdown());
table0.close(); table0.close();
} }
@ -846,14 +846,14 @@ public class TestHCM {
*/ */
@Test @Test
public void testConnectionSameness() throws Exception { public void testConnectionSameness() throws Exception {
HConnection previousConnection = null; Connection previousConnection = null;
for (int i = 0; i < 2; i++) { for (int i = 0; i < 2; i++) {
// set random key to differentiate the connection from previous ones // set random key to differentiate the connection from previous ones
Configuration configuration = TEST_UTIL.getConfiguration(); Configuration configuration = TEST_UTIL.getConfiguration();
configuration.set("some_key", String.valueOf(_randy.nextInt())); configuration.set("some_key", String.valueOf(_randy.nextInt()));
LOG.info("The hash code of the current configuration is: " LOG.info("The hash code of the current configuration is: "
+ configuration.hashCode()); + configuration.hashCode());
HConnection currentConnection = HConnectionManager Connection currentConnection = HConnectionManager
.getConnection(configuration); .getConnection(configuration);
if (previousConnection != null) { if (previousConnection != null) {
assertTrue( assertTrue(
@ -884,7 +884,7 @@ public class TestHCM {
// to set up a session and test runs for a long time. // to set up a session and test runs for a long time.
int maxConnections = Math.min(zkmaxconnections - 1, 20); int maxConnections = Math.min(zkmaxconnections - 1, 20);
List<HConnection> connections = new ArrayList<HConnection>(maxConnections); List<HConnection> connections = new ArrayList<HConnection>(maxConnections);
HConnection previousConnection = null; Connection previousConnection = null;
try { try {
for (int i = 0; i < maxConnections; i++) { for (int i = 0; i < maxConnections; i++) {
// set random key to differentiate the connection from previous ones // set random key to differentiate the connection from previous ones
@ -913,7 +913,7 @@ public class TestHCM {
connections.add(currentConnection); connections.add(currentConnection);
} }
} finally { } finally {
for (HConnection c: connections) { for (Connection c: connections) {
// Clean up connections made so we don't interfere w/ subsequent tests. // Clean up connections made so we don't interfere w/ subsequent tests.
HConnectionManager.deleteConnection(c.getConfiguration()); HConnectionManager.deleteConnection(c.getConfiguration());
} }
@ -927,12 +927,12 @@ public class TestHCM {
configuration.set(HConstants.HBASE_CLIENT_INSTANCE_ID, configuration.set(HConstants.HBASE_CLIENT_INSTANCE_ID,
String.valueOf(_randy.nextInt())); String.valueOf(_randy.nextInt()));
HConnection c1 = HConnectionManager.createConnection(configuration); Connection c1 = ConnectionFactory.createConnection(configuration);
// We create two connections with the same key. // We create two connections with the same key.
HConnection c2 = HConnectionManager.createConnection(configuration); Connection c2 = ConnectionFactory.createConnection(configuration);
HConnection c3 = HConnectionManager.getConnection(configuration); Connection c3 = HConnectionManager.getConnection(configuration);
HConnection c4 = HConnectionManager.getConnection(configuration); Connection c4 = HConnectionManager.getConnection(configuration);
assertTrue(c3 == c4); assertTrue(c3 == c4);
c1.close(); c1.close();
@ -946,7 +946,7 @@ public class TestHCM {
c3.close(); c3.close();
assertTrue(c3.isClosed()); assertTrue(c3.isClosed());
// c3 was removed from the cache // c3 was removed from the cache
HConnection c5 = HConnectionManager.getConnection(configuration); Connection c5 = HConnectionManager.getConnection(configuration);
assertTrue(c5 != c3); assertTrue(c5 != c3);
assertFalse(c2.isClosed()); assertFalse(c2.isClosed());
@ -963,13 +963,13 @@ public class TestHCM {
@Test @Test
public void testCreateConnection() throws Exception { public void testCreateConnection() throws Exception {
Configuration configuration = TEST_UTIL.getConfiguration(); Configuration configuration = TEST_UTIL.getConfiguration();
HConnection c1 = HConnectionManager.createConnection(configuration); Connection c1 = ConnectionFactory.createConnection(configuration);
HConnection c2 = HConnectionManager.createConnection(configuration); Connection c2 = ConnectionFactory.createConnection(configuration);
// created from the same configuration, yet they are different // created from the same configuration, yet they are different
assertTrue(c1 != c2); assertTrue(c1 != c2);
assertTrue(c1.getConfiguration() == c2.getConfiguration()); assertTrue(c1.getConfiguration() == c2.getConfiguration());
// make sure these were not cached // make sure these were not cached
HConnection c3 = HConnectionManager.getConnection(configuration); Connection c3 = HConnectionManager.getConnection(configuration);
assertTrue(c1 != c3); assertTrue(c1 != c3);
assertTrue(c2 != c3); assertTrue(c2 != c3);
} }
@ -1230,7 +1230,7 @@ public class TestHCM {
// Use connection multiple times. // Use connection multiple times.
for (int i = 0; i < 30; i++) { for (int i = 0; i < 30; i++) {
HConnection c1 = null; Connection c1 = null;
try { try {
c1 = ConnectionManager.getConnectionInternal(config); c1 = ConnectionManager.getConnectionInternal(config);
LOG.info("HTable connection " + i + " " + c1); LOG.info("HTable connection " + i + " " + c1);
@ -1273,7 +1273,7 @@ public class TestHCM {
TableName tableName = TableName.valueOf("testConnectionRideOverClusterRestart"); TableName tableName = TableName.valueOf("testConnectionRideOverClusterRestart");
TEST_UTIL.createTable(tableName.getName(), new byte[][] {FAM_NAM}, config).close(); TEST_UTIL.createTable(tableName.getName(), new byte[][] {FAM_NAM}, config).close();
HConnection connection = HConnectionManager.createConnection(config); Connection connection = ConnectionFactory.createConnection(config);
Table table = connection.getTable(tableName); Table table = connection.getTable(tableName);
// this will cache the meta location and table's region location // this will cache the meta location and table's region location

View File

@ -43,10 +43,9 @@ import org.apache.hadoop.hbase.ServerName;
import org.apache.hadoop.hbase.TableName; import org.apache.hadoop.hbase.TableName;
import org.apache.hadoop.hbase.MetaTableAccessor; import org.apache.hadoop.hbase.MetaTableAccessor;
import org.apache.hadoop.hbase.MetaTableAccessor.Visitor; import org.apache.hadoop.hbase.MetaTableAccessor.Visitor;
import org.apache.hadoop.hbase.client.Admin; import org.apache.hadoop.hbase.client.Connection;
import org.apache.hadoop.hbase.client.Delete; import org.apache.hadoop.hbase.client.Delete;
import org.apache.hadoop.hbase.client.HBaseAdmin; import org.apache.hadoop.hbase.client.HBaseAdmin;
import org.apache.hadoop.hbase.client.HConnection;
import org.apache.hadoop.hbase.client.HTable; import org.apache.hadoop.hbase.client.HTable;
import org.apache.hadoop.hbase.client.RegionReplicaUtil; import org.apache.hadoop.hbase.client.RegionReplicaUtil;
import org.apache.hadoop.hbase.client.Result; import org.apache.hadoop.hbase.client.Result;
@ -286,7 +285,7 @@ public class TestMasterOperationsForRegionReplicas {
} }
private void validateNumberOfRowsInMeta(final TableName table, int numRegions, private void validateNumberOfRowsInMeta(final TableName table, int numRegions,
HConnection hConnection) throws IOException { Connection connection) throws IOException {
assert(admin.tableExists(table)); assert(admin.tableExists(table));
final AtomicInteger count = new AtomicInteger(); final AtomicInteger count = new AtomicInteger();
Visitor visitor = new Visitor() { Visitor visitor = new Visitor() {
@ -296,14 +295,14 @@ public class TestMasterOperationsForRegionReplicas {
return true; return true;
} }
}; };
MetaTableAccessor.fullScan(hConnection, visitor); MetaTableAccessor.fullScan(connection, visitor);
assert(count.get() == numRegions); assert(count.get() == numRegions);
} }
private void validateFromSnapshotFromMeta(HBaseTestingUtility util, TableName table, private void validateFromSnapshotFromMeta(HBaseTestingUtility util, TableName table,
int numRegions, int numReplica, HConnection hConnection) throws IOException { int numRegions, int numReplica, Connection connection) throws IOException {
SnapshotOfRegionAssignmentFromMeta snapshot = new SnapshotOfRegionAssignmentFromMeta( SnapshotOfRegionAssignmentFromMeta snapshot = new SnapshotOfRegionAssignmentFromMeta(
hConnection); connection);
snapshot.initialize(); snapshot.initialize();
Map<HRegionInfo, ServerName> regionToServerMap = snapshot.getRegionToRegionServerMap(); Map<HRegionInfo, ServerName> regionToServerMap = snapshot.getRegionToRegionServerMap();
assert(regionToServerMap.size() == numRegions * numReplica + 1); //'1' for the namespace assert(regionToServerMap.size() == numRegions * numReplica + 1); //'1' for the namespace
@ -327,10 +326,10 @@ public class TestMasterOperationsForRegionReplicas {
} }
} }
private void validateSingleRegionServerAssignment(HConnection hConnection, int numRegions, private void validateSingleRegionServerAssignment(Connection connection, int numRegions,
int numReplica) throws IOException { int numReplica) throws IOException {
SnapshotOfRegionAssignmentFromMeta snapshot = new SnapshotOfRegionAssignmentFromMeta( SnapshotOfRegionAssignmentFromMeta snapshot = new SnapshotOfRegionAssignmentFromMeta(
hConnection); connection);
snapshot.initialize(); snapshot.initialize();
Map<HRegionInfo, ServerName> regionToServerMap = snapshot.getRegionToRegionServerMap(); Map<HRegionInfo, ServerName> regionToServerMap = snapshot.getRegionToRegionServerMap();
assertEquals(regionToServerMap.size(), numRegions * numReplica + 1); //'1' for the namespace assertEquals(regionToServerMap.size(), numRegions * numReplica + 1); //'1' for the namespace

View File

@ -37,6 +37,8 @@ import org.apache.hadoop.hbase.HTableDescriptor;
import org.apache.hadoop.hbase.TableName; import org.apache.hadoop.hbase.TableName;
import org.apache.hadoop.hbase.Waiter; import org.apache.hadoop.hbase.Waiter;
import org.apache.hadoop.hbase.client.ClusterConnection; import org.apache.hadoop.hbase.client.ClusterConnection;
import org.apache.hadoop.hbase.client.Connection;
import org.apache.hadoop.hbase.client.ConnectionFactory;
import org.apache.hadoop.hbase.client.HConnection; import org.apache.hadoop.hbase.client.HConnection;
import org.apache.hadoop.hbase.client.HConnectionManager; import org.apache.hadoop.hbase.client.HConnectionManager;
import org.apache.hadoop.hbase.client.RpcRetryingCaller; import org.apache.hadoop.hbase.client.RpcRetryingCaller;
@ -148,7 +150,7 @@ public class TestRegionReplicaReplicationEndpoint {
HTU.deleteTableIfAny(tableNameNoReplicas); HTU.deleteTableIfAny(tableNameNoReplicas);
HTU.createTable(tableNameNoReplicas, HBaseTestingUtility.fam1); HTU.createTable(tableNameNoReplicas, HBaseTestingUtility.fam1);
HConnection connection = HConnectionManager.createConnection(HTU.getConfiguration()); Connection connection = ConnectionFactory.createConnection(HTU.getConfiguration());
Table table = connection.getTable(tableName); Table table = connection.getTable(tableName);
Table tableNoReplicas = connection.getTable(tableNameNoReplicas); Table tableNoReplicas = connection.getTable(tableNameNoReplicas);
@ -235,7 +237,7 @@ public class TestRegionReplicaReplicationEndpoint {
HTU.getHBaseAdmin().createTable(htd); HTU.getHBaseAdmin().createTable(htd);
HConnection connection = HConnectionManager.createConnection(HTU.getConfiguration()); Connection connection = ConnectionFactory.createConnection(HTU.getConfiguration());
Table table = connection.getTable(tableName); Table table = connection.getTable(tableName);
try { try {

View File

@ -37,7 +37,7 @@ import org.apache.hadoop.hbase.HTableDescriptor;
import org.apache.hadoop.hbase.RegionLocations; import org.apache.hadoop.hbase.RegionLocations;
import org.apache.hadoop.hbase.TableName; import org.apache.hadoop.hbase.TableName;
import org.apache.hadoop.hbase.client.ClusterConnection; import org.apache.hadoop.hbase.client.ClusterConnection;
import org.apache.hadoop.hbase.client.HConnectionManager; import org.apache.hadoop.hbase.client.ConnectionFactory;
import org.apache.hadoop.hbase.client.HTable; import org.apache.hadoop.hbase.client.HTable;
import org.apache.hadoop.hbase.client.RpcRetryingCallerFactory; import org.apache.hadoop.hbase.client.RpcRetryingCallerFactory;
import org.apache.hadoop.hbase.coprocessor.BaseWALObserver; import org.apache.hadoop.hbase.coprocessor.BaseWALObserver;
@ -162,7 +162,7 @@ public class TestRegionReplicaReplicationEndpointNoMaster {
// tests replaying the edits to a secondary region replica using the Callable directly // tests replaying the edits to a secondary region replica using the Callable directly
openRegion(HTU, rs0, hriSecondary); openRegion(HTU, rs0, hriSecondary);
ClusterConnection connection = ClusterConnection connection =
(ClusterConnection) HConnectionManager.createConnection(HTU.getConfiguration()); (ClusterConnection) ConnectionFactory.createConnection(HTU.getConfiguration());
//load some data to primary //load some data to primary
HTU.loadNumericRows(table, f, 0, 1000); HTU.loadNumericRows(table, f, 0, 1000);
@ -203,7 +203,7 @@ public class TestRegionReplicaReplicationEndpointNoMaster {
// the region is moved to another location.It tests handling of RME. // the region is moved to another location.It tests handling of RME.
openRegion(HTU, rs0, hriSecondary); openRegion(HTU, rs0, hriSecondary);
ClusterConnection connection = ClusterConnection connection =
(ClusterConnection) HConnectionManager.createConnection(HTU.getConfiguration()); (ClusterConnection) ConnectionFactory.createConnection(HTU.getConfiguration());
//load some data to primary //load some data to primary
HTU.loadNumericRows(table, f, 0, 1000); HTU.loadNumericRows(table, f, 0, 1000);
@ -237,7 +237,7 @@ public class TestRegionReplicaReplicationEndpointNoMaster {
// tests replaying the edits to a secondary region replica using the RRRE.replicate() // tests replaying the edits to a secondary region replica using the RRRE.replicate()
openRegion(HTU, rs0, hriSecondary); openRegion(HTU, rs0, hriSecondary);
ClusterConnection connection = ClusterConnection connection =
(ClusterConnection) HConnectionManager.createConnection(HTU.getConfiguration()); (ClusterConnection) ConnectionFactory.createConnection(HTU.getConfiguration());
RegionReplicaReplicationEndpoint replicator = new RegionReplicaReplicationEndpoint(); RegionReplicaReplicationEndpoint replicator = new RegionReplicaReplicationEndpoint();
ReplicationEndpoint.Context context = mock(ReplicationEndpoint.Context.class); ReplicationEndpoint.Context context = mock(ReplicationEndpoint.Context.class);

View File

@ -49,10 +49,10 @@ import org.apache.hadoop.hbase.TableNotFoundException;
import org.apache.hadoop.hbase.Tag; import org.apache.hadoop.hbase.Tag;
import org.apache.hadoop.hbase.client.Admin; import org.apache.hadoop.hbase.client.Admin;
import org.apache.hadoop.hbase.client.Append; import org.apache.hadoop.hbase.client.Append;
import org.apache.hadoop.hbase.client.Connection;
import org.apache.hadoop.hbase.client.ConnectionFactory;
import org.apache.hadoop.hbase.client.Delete; import org.apache.hadoop.hbase.client.Delete;
import org.apache.hadoop.hbase.client.Get; import org.apache.hadoop.hbase.client.Get;
import org.apache.hadoop.hbase.client.HConnection;
import org.apache.hadoop.hbase.client.HConnectionManager;
import org.apache.hadoop.hbase.client.HTable; import org.apache.hadoop.hbase.client.HTable;
import org.apache.hadoop.hbase.client.Increment; import org.apache.hadoop.hbase.client.Increment;
import org.apache.hadoop.hbase.client.Put; import org.apache.hadoop.hbase.client.Put;
@ -2076,7 +2076,7 @@ public class TestAccessController extends SecureTestUtil {
AccessTestAction deleteTableAction = new AccessTestAction() { AccessTestAction deleteTableAction = new AccessTestAction() {
@Override @Override
public Object run() throws Exception { public Object run() throws Exception {
HConnection unmanagedConnection = HConnectionManager.createConnection(TEST_UTIL.getConfiguration()); Connection unmanagedConnection = ConnectionFactory.createConnection(TEST_UTIL.getConfiguration());
Admin admin = unmanagedConnection.getAdmin(); Admin admin = unmanagedConnection.getAdmin();
try { try {
admin.disableTable(TEST_TABLE.getTableName()); admin.disableTable(TEST_TABLE.getTableName());

View File

@ -69,6 +69,8 @@ import org.apache.hadoop.hbase.ServerName;
import org.apache.hadoop.hbase.TableName; import org.apache.hadoop.hbase.TableName;
import org.apache.hadoop.hbase.MetaTableAccessor; import org.apache.hadoop.hbase.MetaTableAccessor;
import org.apache.hadoop.hbase.client.Admin; import org.apache.hadoop.hbase.client.Admin;
import org.apache.hadoop.hbase.client.Connection;
import org.apache.hadoop.hbase.client.ConnectionFactory;
import org.apache.hadoop.hbase.client.Delete; import org.apache.hadoop.hbase.client.Delete;
import org.apache.hadoop.hbase.client.Durability; import org.apache.hadoop.hbase.client.Durability;
import org.apache.hadoop.hbase.client.Get; import org.apache.hadoop.hbase.client.Get;
@ -2423,7 +2425,7 @@ public class TestHBaseFsck {
HRegionInfo hri = metaLocation.getRegionInfo(); HRegionInfo hri = metaLocation.getRegionInfo();
if (unassign) { if (unassign) {
LOG.info("Undeploying meta region " + hri + " from server " + hsa); LOG.info("Undeploying meta region " + hri + " from server " + hsa);
HConnection unmanagedConnection = HConnectionManager.createConnection(conf); Connection unmanagedConnection = ConnectionFactory.createConnection(conf);
HBaseAdmin admin = (HBaseAdmin) unmanagedConnection.getAdmin(); HBaseAdmin admin = (HBaseAdmin) unmanagedConnection.getAdmin();
try { try {
undeployRegion(admin, hsa, hri); undeployRegion(admin, hsa, hri);

View File

@ -29,10 +29,9 @@ import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.fs.FileSystem; import org.apache.hadoop.fs.FileSystem;
import org.apache.hadoop.fs.Path; import org.apache.hadoop.fs.Path;
import org.apache.hadoop.hbase.*; import org.apache.hadoop.hbase.*;
import org.apache.hadoop.hbase.MetaTableAccessor;
import org.apache.hadoop.hbase.client.Admin; import org.apache.hadoop.hbase.client.Admin;
import org.apache.hadoop.hbase.client.Connection;
import org.apache.hadoop.hbase.client.HBaseAdmin; import org.apache.hadoop.hbase.client.HBaseAdmin;
import org.apache.hadoop.hbase.client.HConnection;
import org.apache.hadoop.hbase.client.HConnectionManager; import org.apache.hadoop.hbase.client.HConnectionManager;
import org.apache.hadoop.hbase.client.Put; import org.apache.hadoop.hbase.client.Put;
import org.apache.hadoop.hbase.client.Durability; import org.apache.hadoop.hbase.client.Durability;
@ -115,17 +114,17 @@ public class TestMergeTable {
LOG.info("Starting mini hbase cluster"); LOG.info("Starting mini hbase cluster");
UTIL.startMiniHBaseCluster(1, 1); UTIL.startMiniHBaseCluster(1, 1);
Configuration c = new Configuration(UTIL.getConfiguration()); Configuration c = new Configuration(UTIL.getConfiguration());
HConnection hConnection = HConnectionManager.getConnection(c); Connection connection = HConnectionManager.getConnection(c);
List<HRegionInfo> originalTableRegions = List<HRegionInfo> originalTableRegions =
MetaTableAccessor.getTableRegions(hConnection, desc.getTableName()); MetaTableAccessor.getTableRegions(connection, desc.getTableName());
LOG.info("originalTableRegions size=" + originalTableRegions.size() + LOG.info("originalTableRegions size=" + originalTableRegions.size() +
"; " + originalTableRegions); "; " + originalTableRegions);
Admin admin = new HBaseAdmin(c); Admin admin = new HBaseAdmin(c);
admin.disableTable(desc.getTableName()); admin.disableTable(desc.getTableName());
HMerge.merge(c, FileSystem.get(c), desc.getTableName()); HMerge.merge(c, FileSystem.get(c), desc.getTableName());
List<HRegionInfo> postMergeTableRegions = List<HRegionInfo> postMergeTableRegions =
MetaTableAccessor.getTableRegions(hConnection, desc.getTableName()); MetaTableAccessor.getTableRegions(connection, desc.getTableName());
LOG.info("postMergeTableRegions size=" + postMergeTableRegions.size() + LOG.info("postMergeTableRegions size=" + postMergeTableRegions.size() +
"; " + postMergeTableRegions); "; " + postMergeTableRegions);
assertTrue("originalTableRegions=" + originalTableRegions.size() + assertTrue("originalTableRegions=" + originalTableRegions.size() +