@InterfaceAudience.Private public class MetaTableAccessor extends Object
hbase:meta
.
Some of the methods of this class take ZooKeeperWatcher as a param. The only reason
for this is because when used on client-side (like from HBaseAdmin), we want to use
short-living connection (opened before each operation, closed right after), while
when used on HM or HRS (like in AssignmentManager) we want permanent connection.Modifier and Type | Class and Description |
---|---|
(package private) static class |
MetaTableAccessor.CollectAllVisitor
Collects all returned.
|
(package private) static class |
MetaTableAccessor.CollectingVisitor<T>
A
MetaTableAccessor.Visitor that collects content out of passed Result . |
static interface |
MetaTableAccessor.Visitor
Implementations 'visit' a catalog table row.
|
Modifier and Type | Field and Description |
---|---|
private static org.apache.commons.logging.Log |
LOG |
(package private) static byte[] |
META_REGION_PREFIX |
protected static char |
META_REPLICA_ID_DELIMITER
The delimiter for meta columns for replicaIds > 0
|
private static Pattern |
SERVER_COLUMN_PATTERN
A regex for parsing server columns from meta.
|
Constructor and Description |
---|
MetaTableAccessor() |
Modifier and Type | Method and Description |
---|---|
static void |
addDaughter(Connection connection,
HRegionInfo regionInfo,
ServerName sn,
long openSeqNum)
Adds a daughter region entry to meta.
|
static Put |
addDaughtersToPut(Put put,
HRegionInfo splitA,
HRegionInfo splitB)
Adds split daughters to the Put
|
static Put |
addEmptyLocation(Put p,
int replicaId) |
static Put |
addLocation(Put p,
ServerName sn,
long openSeqNum,
long time,
int replicaId) |
private static Put |
addRegionInfo(Put p,
HRegionInfo hri) |
static void |
addRegionsToMeta(Connection connection,
List<HRegionInfo> regionInfos,
int regionReplication)
Adds a hbase:meta row for each of the specified new regions.
|
static void |
addRegionsToMeta(Connection connection,
List<HRegionInfo> regionInfos,
int regionReplication,
long ts)
Adds a hbase:meta row for each of the specified new regions.
|
static void |
addRegionToMeta(Connection connection,
HRegionInfo regionInfo)
Adds a hbase:meta row for the specified new region.
|
static void |
addRegionToMeta(Connection connection,
HRegionInfo regionInfo,
HRegionInfo splitA,
HRegionInfo splitB)
Adds a (single) hbase:meta row for the specified new region and its daughters.
|
static void |
addRegionToMeta(Table meta,
HRegionInfo regionInfo)
Adds a hbase:meta row for the specified new region to the given catalog table.
|
static void |
addRegionToMeta(Table meta,
HRegionInfo regionInfo,
HRegionInfo splitA,
HRegionInfo splitB)
Adds a (single) hbase:meta row for the specified new region and its daughters.
|
(package private) static void |
deleteFromMetaTable(Connection connection,
Delete d)
Delete the passed
d from the hbase:meta table. |
static void |
deleteFromMetaTable(Connection connection,
List<Delete> deletes)
Delete the passed
deletes from the hbase:meta table. |
static void |
deleteMergeQualifiers(Connection connection,
HRegionInfo mergedRegion)
Deletes merge qualifiers for the specified merged region.
|
static void |
deleteRegion(Connection connection,
HRegionInfo regionInfo)
Deletes the specified region from META.
|
static void |
deleteRegions(Connection connection,
List<HRegionInfo> regionsInfo)
Deletes the specified regions from META.
|
static void |
deleteRegions(Connection connection,
List<HRegionInfo> regionsInfo,
long ts)
Deletes the specified regions from META.
|
static List<Result> |
fullScan(Connection connection)
Performs a full scan of
hbase:meta . |
static void |
fullScan(Connection connection,
MetaTableAccessor.Visitor visitor)
Performs a full scan of
hbase:meta . |
static void |
fullScan(Connection connection,
MetaTableAccessor.Visitor visitor,
byte[] startrow)
Performs a full scan of a catalog table.
|
static void |
fullScanMetaAndPrint(Connection connection) |
static List<Result> |
fullScanOfMeta(Connection connection)
Performs a full scan of a
hbase:meta table. |
private static Result |
get(Table t,
Get g) |
static PairOfSameType<HRegionInfo> |
getDaughterRegions(Result data)
Returns the daughter regions by reading the corresponding columns of the catalog table
Result.
|
protected static byte[] |
getFamily()
Returns the column family used for meta columns.
|
static HRegionInfo |
getHRegionInfo(Result data)
Returns HRegionInfo object from the column
HConstants.CATALOG_FAMILY:HConstants.REGIONINFO_QUALIFIER of the catalog
table Result.
|
private static HRegionInfo |
getHRegionInfo(Result r,
byte[] qualifier)
Returns the HRegionInfo object from the column
HConstants.CATALOG_FAMILY and
qualifier of the catalog table result. |
(package private) static List<HRegionInfo> |
getListOfHRegionInfos(List<Pair<HRegionInfo,ServerName>> pairs) |
static PairOfSameType<HRegionInfo> |
getMergeRegions(Result data)
Returns the merge regions by reading the corresponding columns of the catalog table
Result.
|
(package private) static Table |
getMetaHTable(Connection connection)
Callers should call close on the returned
Table instance. |
static byte[] |
getMetaKeyForRegion(HRegionInfo regionInfo)
Returns the row key to use for this regionInfo
|
static Pair<HRegionInfo,ServerName> |
getRegion(Connection connection,
byte[] regionName)
Deprecated.
use
getRegionLocation(Connection, byte[]) instead |
static int |
getRegionCount(org.apache.hadoop.conf.Configuration c,
String tableName)
Deprecated.
|
static int |
getRegionCount(org.apache.hadoop.conf.Configuration c,
TableName tableName)
Count regions in
hbase:meta for passed table. |
static int |
getRegionCount(Connection connection,
TableName tableName)
Count regions in
hbase:meta for passed table. |
protected static byte[] |
getRegionInfoColumn()
Returns the column qualifier for serialized region info
|
static HRegionLocation |
getRegionLocation(Connection connection,
byte[] regionName)
Returns the HRegionLocation from meta for the given region
|
static HRegionLocation |
getRegionLocation(Connection connection,
HRegionInfo regionInfo)
Returns the HRegionLocation from meta for the given region
|
private static HRegionLocation |
getRegionLocation(Result r,
HRegionInfo regionInfo,
int replicaId)
Returns the HRegionLocation parsed from the given meta row Result
for the given regionInfo and replicaId.
|
static RegionLocations |
getRegionLocations(Result r)
Returns an HRegionLocationList extracted from the result.
|
static Result |
getRegionResult(Connection connection,
byte[] regionName)
Gets the result in hbase:meta for the specified region.
|
static Pair<HRegionInfo,HRegionInfo> |
getRegionsFromMergeQualifier(Connection connection,
byte[] regionName)
Get regions from the merge qualifier of the specified merged region
|
static Scan |
getScanForTableName(TableName tableName)
This method creates a Scan object that will only scan catalog rows that
belong to the specified table.
|
static byte[] |
getSeqNumColumn(int replicaId)
Returns the column qualifier for seqNum column for replicaId
|
private static long |
getSeqNumDuringOpen(Result r,
int replicaId)
The latest seqnum that the server writing to meta observed when opening the region.
|
static byte[] |
getServerColumn(int replicaId)
Returns the column qualifier for server column for replicaId
|
private static ServerName |
getServerName(Result r,
int replicaId)
Returns a
ServerName from catalog table Result . |
static NavigableMap<HRegionInfo,Result> |
getServerUserRegions(Connection connection,
ServerName serverName) |
static byte[] |
getStartCodeColumn(int replicaId)
Returns the column qualifier for server start code column for replicaId
|
static List<HRegionInfo> |
getTableRegions(ZooKeeperWatcher zkw,
Connection connection,
TableName tableName)
Gets all of the regions of the specified table.
|
static List<HRegionInfo> |
getTableRegions(ZooKeeperWatcher zkw,
Connection connection,
TableName tableName,
boolean excludeOfflinedSplitParents)
Gets all of the regions of the specified table.
|
static List<Pair<HRegionInfo,ServerName>> |
getTableRegionsAndLocations(ZooKeeperWatcher zkw,
Connection connection,
TableName tableName) |
static List<Pair<HRegionInfo,ServerName>> |
getTableRegionsAndLocations(ZooKeeperWatcher zkw,
Connection connection,
TableName tableName,
boolean excludeOfflinedSplitParents) |
(package private) static byte[] |
getTableStartRowForMeta(TableName tableName) |
(package private) static boolean |
isInsideTable(HRegionInfo current,
TableName tableName) |
static Delete |
makeDeleteFromRegionInfo(HRegionInfo regionInfo)
Generates and returns a Delete containing the region info for the catalog
table
|
static Delete |
makeDeleteFromRegionInfo(HRegionInfo regionInfo,
long ts)
Generates and returns a Delete containing the region info for the catalog
table
|
static Put |
makePutFromRegionInfo(HRegionInfo regionInfo)
Generates and returns a Put containing the region into for the catalog table
|
static Put |
makePutFromRegionInfo(HRegionInfo regionInfo,
long ts)
Generates and returns a Put containing the region into for the catalog table
|
static void |
mergeRegions(Connection connection,
HRegionInfo mergedRegion,
HRegionInfo regionA,
HRegionInfo regionB,
ServerName sn,
int regionReplication,
long masterSystemTime)
Merge the two regions into one in an atomic operation.
|
private static void |
multiMutate(Table table,
byte[] row,
Mutation... mutations)
Performs an atomic multi-Mutate operation against the given table.
|
static void |
mutateMetaTable(Connection connection,
List<Mutation> mutations)
Execute the passed
mutations against hbase:meta table. |
static void |
mutateRegions(Connection connection,
List<HRegionInfo> regionsToRemove,
List<HRegionInfo> regionsToAdd)
Adds and Removes the specified regions from hbase:meta
|
static void |
overwriteRegions(Connection connection,
List<HRegionInfo> regionInfos,
int regionReplication)
Overwrites the specified regions from hbase:meta
|
protected static HRegionInfo |
parseRegionInfoFromRegionName(byte[] regionName)
Returns an HRI parsed from this regionName.
|
(package private) static int |
parseReplicaIdFromServerColumn(byte[] serverColumn)
Parses the replicaId from the server column qualifier.
|
private static void |
put(Table t,
Put p) |
static void |
putsToMetaTable(Connection connection,
List<Put> ps)
Put the passed
ps to the hbase:meta table. |
(package private) static void |
putToMetaTable(Connection connection,
Put p)
Put the passed
p to the hbase:meta table. |
static void |
removeRegionReplicasFromMeta(Set<byte[]> metaRows,
int replicaIndexToDeleteFrom,
int numReplicasToRemove,
Connection connection)
Deletes some replica columns corresponding to replicas for the passed rows
|
static void |
splitRegion(Connection connection,
HRegionInfo parent,
HRegionInfo splitA,
HRegionInfo splitB,
ServerName sn,
int regionReplication)
Splits the region into two in an atomic operation.
|
static boolean |
tableExists(Connection connection,
TableName tableName)
Checks if the specified table exists.
|
private static void |
updateLocation(Connection connection,
HRegionInfo regionInfo,
ServerName sn,
long openSeqNum,
long masterSystemTime)
Updates the location of the specified region to be the specified server.
|
static void |
updateRegionLocation(Connection connection,
HRegionInfo regionInfo,
ServerName sn,
long openSeqNum,
long masterSystemTime)
Updates the location of the specified region in hbase:meta to be the specified
server hostname and startcode.
|
private static final org.apache.commons.logging.Log LOG
static final byte[] META_REGION_PREFIX
protected static final char META_REPLICA_ID_DELIMITER
private static final Pattern SERVER_COLUMN_PATTERN
public static List<Result> fullScanOfMeta(Connection connection) throws IOException
hbase:meta
table.Result
IOException
public static void fullScan(Connection connection, MetaTableAccessor.Visitor visitor) throws IOException
hbase:meta
.connection
- connection we're usingvisitor
- Visitor invoked against each row.IOException
public static List<Result> fullScan(Connection connection) throws IOException
hbase:meta
.connection
- connection we're usingResult
IOException
static Table getMetaHTable(Connection connection) throws IOException
Table
instance.connection
- connection we're using to access MetaTable
for hbase:meta
IOException
private static Result get(Table t, Get g) throws IOException
t
- Table to use (will be closed when done).g
- Get to runIOException
@Deprecated public static Pair<HRegionInfo,ServerName> getRegion(Connection connection, byte[] regionName) throws IOException
getRegionLocation(Connection, byte[])
insteadconnection
- connection we're usingregionName
- Region to lookup.regionName
IOException
public static HRegionLocation getRegionLocation(Connection connection, byte[] regionName) throws IOException
connection
- connection we're usingregionName
- region we're looking forIOException
public static HRegionLocation getRegionLocation(Connection connection, HRegionInfo regionInfo) throws IOException
connection
- connection we're usingregionInfo
- region informationIOException
public static byte[] getMetaKeyForRegion(HRegionInfo regionInfo)
protected static HRegionInfo parseRegionInfoFromRegionName(byte[] regionName) throws IOException
IOException
public static Result getRegionResult(Connection connection, byte[] regionName) throws IOException
connection
- connection we're usingregionName
- region we're looking forIOException
public static Pair<HRegionInfo,HRegionInfo> getRegionsFromMergeQualifier(Connection connection, byte[] regionName) throws IOException
IOException
public static boolean tableExists(Connection connection, TableName tableName) throws IOException
connection
- connection we're usingtableName
- table to checkIOException
public static List<HRegionInfo> getTableRegions(ZooKeeperWatcher zkw, Connection connection, TableName tableName) throws IOException
zkw
- zookeeper connection to access meta tableconnection
- connection we're usingtableName
- table we're looking forHRegionInfo
.IOException
public static List<HRegionInfo> getTableRegions(ZooKeeperWatcher zkw, Connection connection, TableName tableName, boolean excludeOfflinedSplitParents) throws IOException
zkw
- zookeeper connection to access meta tableconnection
- connection we're usingtableName
- table we're looking forexcludeOfflinedSplitParents
- If true, do not include offlined split
parents in the return.HRegionInfo
.IOException
static List<HRegionInfo> getListOfHRegionInfos(List<Pair<HRegionInfo,ServerName>> pairs)
static boolean isInsideTable(HRegionInfo current, TableName tableName)
current
- region of current table we're working withtableName
- table we're checking againstcurrent
tablename is equal to
tableName
static byte[] getTableStartRowForMeta(TableName tableName)
tableName
- table we're working withhbase:meta
when passed a
tableName
; returns <tableName&rt; <,&rt; <,&rt;public static Scan getScanForTableName(TableName tableName)
tableName
- bytes of table's namepublic static List<Pair<HRegionInfo,ServerName>> getTableRegionsAndLocations(ZooKeeperWatcher zkw, Connection connection, TableName tableName) throws IOException
zkw
- zookeeper connection to access meta tableconnection
- connection we're usingtableName
- table we're looking forIOException
public static List<Pair<HRegionInfo,ServerName>> getTableRegionsAndLocations(ZooKeeperWatcher zkw, Connection connection, TableName tableName, boolean excludeOfflinedSplitParents) throws IOException
zkw
- ZooKeeperWatcher instance we're using to get hbase:meta locationconnection
- connection we're usingtableName
- table to work withIOException
public static NavigableMap<HRegionInfo,Result> getServerUserRegions(Connection connection, ServerName serverName) throws IOException
connection
- connection we're usingserverName
- server whose regions we're interested inIOException
public static void fullScanMetaAndPrint(Connection connection) throws IOException
IOException
public static void fullScan(Connection connection, MetaTableAccessor.Visitor visitor, byte[] startrow) throws IOException
connection
- connection we're usingvisitor
- Visitor invoked against each row.startrow
- Where to start the scan. Pass null if want to begin scan
at first row.
hbase:meta
, the default (pass false to scan hbase:meta)IOException
protected static byte[] getFamily()
protected static byte[] getRegionInfoColumn()
public static byte[] getServerColumn(int replicaId)
replicaId
- the replicaId of the regionpublic static byte[] getStartCodeColumn(int replicaId)
replicaId
- the replicaId of the regionpublic static byte[] getSeqNumColumn(int replicaId)
replicaId
- the replicaId of the regionstatic int parseReplicaIdFromServerColumn(byte[] serverColumn)
serverColumn
- the column qualifierprivate static ServerName getServerName(Result r, int replicaId)
ServerName
from catalog table Result
.r
- Result to pull fromprivate static long getSeqNumDuringOpen(Result r, int replicaId)
getServerName(Result, int)
was written.r
- Result to pull the seqNum frompublic static RegionLocations getRegionLocations(Result r)
private static HRegionLocation getRegionLocation(Result r, HRegionInfo regionInfo, int replicaId)
r
- the meta row resultregionInfo
- RegionInfo for default replicareplicaId
- the replicaId for the HRegionLocationpublic static HRegionInfo getHRegionInfo(Result data)
data
- a Result object from the catalog table scanprivate static HRegionInfo getHRegionInfo(Result r, byte[] qualifier)
HConstants.CATALOG_FAMILY
and
qualifier
of the catalog table result.r
- a Result object from the catalog table scanqualifier
- Column family qualifierpublic static PairOfSameType<HRegionInfo> getDaughterRegions(Result data)
data
- a Result object from the catalog table scanpublic static PairOfSameType<HRegionInfo> getMergeRegions(Result data)
data
- a Result object from the catalog table scan@Deprecated public static int getRegionCount(org.apache.hadoop.conf.Configuration c, String tableName) throws IOException
hbase:meta
for passed table.c
- Configuration objecttableName
- table name to count regions fortableName
IOException
public static int getRegionCount(org.apache.hadoop.conf.Configuration c, TableName tableName) throws IOException
hbase:meta
for passed table.c
- Configuration objecttableName
- table name to count regions fortableName
IOException
public static int getRegionCount(Connection connection, TableName tableName) throws IOException
hbase:meta
for passed table.connection
- Connection objecttableName
- table name to count regions fortableName
IOException
public static Put makePutFromRegionInfo(HRegionInfo regionInfo) throws IOException
IOException
public static Put makePutFromRegionInfo(HRegionInfo regionInfo, long ts) throws IOException
IOException
public static Delete makeDeleteFromRegionInfo(HRegionInfo regionInfo)
public static Delete makeDeleteFromRegionInfo(HRegionInfo regionInfo, long ts)
public static Put addDaughtersToPut(Put put, HRegionInfo splitA, HRegionInfo splitB)
static void putToMetaTable(Connection connection, Put p) throws IOException
p
to the hbase:meta
table.connection
- connection we're usingp
- Put to add to hbase:metaIOException
private static void put(Table t, Put p) throws IOException
t
- Table to use (will be closed when done).p
- put to makeIOException
public static void putsToMetaTable(Connection connection, List<Put> ps) throws IOException
ps
to the hbase:meta
table.connection
- connection we're usingps
- Put to add to hbase:metaIOException
static void deleteFromMetaTable(Connection connection, Delete d) throws IOException
d
from the hbase:meta
table.connection
- connection we're usingd
- Delete to add to hbase:metaIOException
public static void deleteFromMetaTable(Connection connection, List<Delete> deletes) throws IOException
deletes
from the hbase:meta
table.connection
- connection we're usingdeletes
- Deletes to add to hbase:meta This list should support #remove.IOException
public static void removeRegionReplicasFromMeta(Set<byte[]> metaRows, int replicaIndexToDeleteFrom, int numReplicasToRemove, Connection connection) throws IOException
metaRows
- rows in hbase:metareplicaIndexToDeleteFrom
- the replica ID we would start deleting fromnumReplicasToRemove
- how many replicas to removeconnection
- connection we're using to access meta tableIOException
public static void mutateMetaTable(Connection connection, List<Mutation> mutations) throws IOException
mutations
against hbase:meta
table.connection
- connection we're usingmutations
- Puts and Deletes to execute on hbase:metaIOException
public static void addRegionToMeta(Connection connection, HRegionInfo regionInfo) throws IOException
connection
- connection we're usingregionInfo
- region informationIOException
- if problem connecting or updating metapublic static void addRegionToMeta(Table meta, HRegionInfo regionInfo) throws IOException
meta
- the Table for METAregionInfo
- region informationIOException
- if problem connecting or updating metapublic static void addRegionToMeta(Table meta, HRegionInfo regionInfo, HRegionInfo splitA, HRegionInfo splitB) throws IOException
splitRegion(Connection, HRegionInfo, HRegionInfo, HRegionInfo, ServerName, int)
if you want to do that.meta
- the Table for METAregionInfo
- region informationsplitA
- first split daughter of the parent regionInfosplitB
- second split daughter of the parent regionInfoIOException
- if problem connecting or updating metapublic static void addRegionToMeta(Connection connection, HRegionInfo regionInfo, HRegionInfo splitA, HRegionInfo splitB) throws IOException
splitRegion(Connection, HRegionInfo, HRegionInfo, HRegionInfo, ServerName, int)
if you want to do that.connection
- connection we're usingregionInfo
- region informationsplitA
- first split daughter of the parent regionInfosplitB
- second split daughter of the parent regionInfoIOException
- if problem connecting or updating metapublic static void addRegionsToMeta(Connection connection, List<HRegionInfo> regionInfos, int regionReplication) throws IOException
connection
- connection we're usingregionInfos
- region information listIOException
- if problem connecting or updating metapublic static void addRegionsToMeta(Connection connection, List<HRegionInfo> regionInfos, int regionReplication, long ts) throws IOException
connection
- connection we're usingregionInfos
- region information listregionReplication
- ts
- desired timestampIOException
- if problem connecting or updating metapublic static void addDaughter(Connection connection, HRegionInfo regionInfo, ServerName sn, long openSeqNum) throws NotAllMetaRegionsOnlineException, IOException
regionInfo
- the region to putsn
- the location of the regionopenSeqNum
- the latest sequence number obtained when the region was openNotAllMetaRegionsOnlineException
IOException
public static void mergeRegions(Connection connection, HRegionInfo mergedRegion, HRegionInfo regionA, HRegionInfo regionB, ServerName sn, int regionReplication, long masterSystemTime) throws IOException
connection
- connection we're usingmergedRegion
- the merged regionregionA
- regionB
- sn
- the location of the regionmasterSystemTime
- IOException
public static void splitRegion(Connection connection, HRegionInfo parent, HRegionInfo splitA, HRegionInfo splitB, ServerName sn, int regionReplication) throws IOException
connection
- connection we're usingparent
- the parent region which is splitsplitA
- Split daughter region AsplitB
- Split daughter region Asn
- the location of the regionIOException
private static void multiMutate(Table table, byte[] row, Mutation... mutations) throws IOException
IOException
public static void updateRegionLocation(Connection connection, HRegionInfo regionInfo, ServerName sn, long openSeqNum, long masterSystemTime) throws IOException
Uses passed catalog tracker to get a connection to the server hosting hbase:meta and makes edits to that region.
connection
- connection we're usingregionInfo
- region to update location ofopenSeqNum
- the latest sequence number obtained when the region was opensn
- Server namemasterSystemTime
- wall clock time from master if passed in the open region RPC or -1IOException
private static void updateLocation(Connection connection, HRegionInfo regionInfo, ServerName sn, long openSeqNum, long masterSystemTime) throws IOException
Connects to the specified server which should be hosting the specified catalog region name to perform the edit.
connection
- connection we're usingregionInfo
- region to update location ofsn
- Server nameopenSeqNum
- the latest sequence number obtained when the region was openmasterSystemTime
- wall clock time from master if passed in the open region RPC or -1IOException
- In particular could throw ConnectException
if the server is down on other end.public static void deleteRegion(Connection connection, HRegionInfo regionInfo) throws IOException
connection
- connection we're usingregionInfo
- region to be deleted from METAIOException
public static void deleteRegions(Connection connection, List<HRegionInfo> regionsInfo, long ts) throws IOException
connection
- connection we're usingregionsInfo
- list of regions to be deleted from METAts
- desired timestampIOException
public static void deleteRegions(Connection connection, List<HRegionInfo> regionsInfo) throws IOException
connection
- connection we're usingregionsInfo
- list of regions to be deleted from METAIOException
public static void mutateRegions(Connection connection, List<HRegionInfo> regionsToRemove, List<HRegionInfo> regionsToAdd) throws IOException
connection
- connection we're usingregionsToRemove
- list of regions to be deleted from METAregionsToAdd
- list of regions to be added to METAIOException
public static void overwriteRegions(Connection connection, List<HRegionInfo> regionInfos, int regionReplication) throws IOException
connection
- connection we're usingregionInfos
- list of regions to be added to METAIOException
public static void deleteMergeQualifiers(Connection connection, HRegionInfo mergedRegion) throws IOException
connection
- connection we're usingmergedRegion
- IOException
private static Put addRegionInfo(Put p, HRegionInfo hri) throws IOException
IOException
public static Put addLocation(Put p, ServerName sn, long openSeqNum, long time, int replicaId)
Copyright © 2007–2019 The Apache Software Foundation. All rights reserved.