@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 |
---|---|
static interface |
MetaTableAccessor.CloseableVisitor
Implementations 'visit' a catalog table row but with close() at the end.
|
static class |
MetaTableAccessor.DefaultVisitorBase
A Visitor that skips offline regions and split parents
|
static class |
MetaTableAccessor.QueryType |
static class |
MetaTableAccessor.TableVisitorBase
A Visitor for a table.
|
static interface |
MetaTableAccessor.Visitor
Implementations 'visit' a catalog table row.
|
Modifier and Type | Field and Description |
---|---|
protected static char |
META_REPLICA_ID_DELIMITER
The delimiter for meta columns for replicaIds > 0
|
Constructor and Description |
---|
MetaTableAccessor() |
Modifier and Type | Method and Description |
---|---|
static Put |
addDaughtersToPut(Put put,
RegionInfo splitA,
RegionInfo 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) |
static Put |
addRegionInfo(Put p,
RegionInfo hri) |
static void |
addRegionsToMeta(Connection connection,
List<RegionInfo> regionInfos,
int regionReplication)
Adds a hbase:meta row for each of the specified new regions.
|
static void |
addRegionsToMeta(Connection connection,
List<RegionInfo> regionInfos,
int regionReplication,
long ts)
Adds a hbase:meta row for each of the specified new regions.
|
static void |
addRegionToMeta(Connection connection,
RegionInfo regionInfo)
Adds a hbase:meta row for the specified new region.
|
static Put |
addSequenceNum(Put p,
long openSeqNum,
long time,
int replicaId) |
static void |
addSpiltsToParent(Connection connection,
RegionInfo regionInfo,
RegionInfo splitA,
RegionInfo splitB)
Adds daughter region infos to hbase:meta row for the specified region.
|
static NavigableMap<RegionInfo,ServerName> |
allTableRegions(Connection connection,
TableName tableName)
Deprecated.
use
getTableRegionsAndLocations(org.apache.hadoop.hbase.client.Connection, org.apache.hadoop.hbase.TableName) , region can have multiple locations |
static void |
deleteFromMetaTable(Connection connection,
List<Delete> deletes)
Delete the passed
deletes from the hbase:meta table. |
static void |
deleteMergeQualifiers(Connection connection,
RegionInfo mergedRegion)
Deletes merge qualifiers for the specified merged region.
|
static void |
deleteRegion(Connection connection,
RegionInfo regionInfo)
Deletes the specified region from META.
|
static void |
deleteRegions(Connection connection,
List<RegionInfo> regionsInfo)
Deletes the specified regions from META.
|
static void |
deleteRegions(Connection connection,
List<RegionInfo> regionsInfo,
long ts)
Deletes the specified regions from META.
|
static void |
deleteTableState(Connection connection,
TableName table)
Remove state for table from meta
|
static List<Result> |
fullScan(Connection connection,
MetaTableAccessor.QueryType type)
Performs a full scan of
hbase:meta . |
static void |
fullScanMetaAndPrint(Connection connection) |
static List<Result> |
fullScanRegions(Connection connection)
Performs a full scan of
hbase:meta for regions. |
static void |
fullScanRegions(Connection connection,
MetaTableAccessor.Visitor visitor)
Performs a full scan of
hbase:meta for regions. |
static void |
fullScanTables(Connection connection,
MetaTableAccessor.Visitor visitor)
Performs a full scan of
hbase:meta for tables. |
static Map<String,List<Long>> |
getAllBarriers(Connection connection)
Get all barriers in all regions.
|
static List<RegionInfo> |
getAllRegions(Connection connection,
boolean excludeOfflinedSplitParents)
Lists all of the regions currently in META.
|
protected static byte[] |
getCatalogFamily()
Returns the column family used for meta columns.
|
static RegionInfo |
getClosestRegionInfo(Connection connection,
TableName tableName,
byte[] row) |
static PairOfSameType<RegionInfo> |
getDaughterRegions(Result data)
Returns the daughter regions by reading the corresponding columns of the catalog table
Result.
|
static PairOfSameType<RegionInfo> |
getMergeRegions(Result data)
Returns the merge regions by reading the corresponding columns of the catalog table
Result.
|
static Table |
getMetaHTable(Connection connection)
Callers should call close on the returned
Table instance. |
static byte[] |
getMetaKeyForRegion(RegionInfo regionInfo)
Returns the row key to use for this regionInfo
|
static Pair<RegionInfo,ServerName> |
getRegion(Connection connection,
byte[] regionName)
Deprecated.
use
getRegionLocation(Connection, byte[]) instead |
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. |
static RegionInfo |
getRegionInfo(Result data)
Returns RegionInfo object from the column
HConstants.CATALOG_FAMILY:HConstants.REGIONINFO_QUALIFIER of the catalog
table Result.
|
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,
RegionInfo regionInfo)
Returns the HRegionLocation from meta for the given region
|
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<RegionInfo,RegionInfo> |
getRegionsFromMergeQualifier(Connection connection,
byte[] regionName)
Get regions from the merge qualifier of the specified merged region
|
protected static byte[] |
getRegionStateColumn()
Returns the column qualifier for serialized region state
|
static List<Long> |
getReplicationBarriers(Connection connection,
byte[] encodedRegionName)
Get replication barriers for all peers in a region.
|
static Map<String,Long> |
getReplicationPositionForAllPeer(Connection connection,
byte[] encodedRegionName)
Get replication positions for all peers in a region.
|
static long |
getReplicationPositionForOnePeer(Connection connection,
byte[] encodedRegionName,
String peerId)
Get replication position for a peer in a region.
|
static Scan |
getScanForTableName(Connection connection,
TableName tableName)
Deprecated.
|
static byte[] |
getSeqNumColumn(int replicaId)
Returns the column qualifier for seqNum column for replicaId
|
static String |
getSerialReplicationDaughterRegion(Connection connection,
byte[] encodedName)
Get daughter region(s) for a region, only used in serial replication.
|
static String |
getSerialReplicationParentRegion(Connection connection,
byte[] encodedName)
Get parent region(s) for a region, only used in serial replication.
|
static String |
getSerialReplicationTableName(Connection connection,
byte[] encodedName)
Get the table name for a region, only used in serial replication.
|
static byte[] |
getServerColumn(int replicaId)
Returns the column qualifier for server column for replicaId
|
static ServerName |
getServerName(Result r,
int replicaId)
Returns a
ServerName from catalog table Result . |
static NavigableMap<RegionInfo,Result> |
getServerUserRegions(Connection connection,
ServerName serverName) |
static byte[] |
getStartCodeColumn(int replicaId)
Returns the column qualifier for server start code column for replicaId
|
protected static byte[] |
getTableFamily()
Returns the column family used for table columns.
|
static List<RegionInfo> |
getTableRegions(Connection connection,
TableName tableName)
Gets all of the regions of the specified table.
|
static List<RegionInfo> |
getTableRegions(Connection connection,
TableName tableName,
boolean excludeOfflinedSplitParents)
Gets all of the regions of the specified table.
|
static List<Pair<RegionInfo,ServerName>> |
getTableRegionsAndLocations(Connection connection,
TableName tableName)
Do not use this method to get meta table regions, use methods in MetaTableLocator instead.
|
static List<Pair<RegionInfo,ServerName>> |
getTableRegionsAndLocations(Connection connection,
TableName tableName,
boolean excludeOfflinedSplitParents)
Do not use this method to get meta table regions, use methods in MetaTableLocator instead.
|
static byte[] |
getTableStartRowForMeta(TableName tableName,
MetaTableAccessor.QueryType type) |
static TableState |
getTableState(Connection conn,
TableName tableName)
Fetch table state for given table from META table
|
static TableState |
getTableState(Result r)
Decode table state from META Result.
|
protected static byte[] |
getTableStateColumn()
Returns the column qualifier for serialized table state
|
static Map<TableName,TableState> |
getTableStates(Connection conn)
Fetch table states from META table
|
static byte[] |
getTableStopRowForMeta(TableName tableName,
MetaTableAccessor.QueryType type) |
static Put |
makeBarrierPut(byte[] encodedRegionName,
long seq,
byte[] tableName) |
static Put |
makeDaughterPut(byte[] encodedRegionName,
byte[] value) |
static Delete |
makeDeleteFromRegionInfo(RegionInfo regionInfo)
Generates and returns a Delete containing the region info for the catalog
table
|
static Delete |
makeDeleteFromRegionInfo(RegionInfo regionInfo,
long ts)
Generates and returns a Delete containing the region info for the catalog
table
|
static Put |
makeParentPut(byte[] encodedRegionName,
byte[] value) |
static Put |
makePutFromRegionInfo(RegionInfo regionInfo)
Generates and returns a Put containing the region into for the catalog table
|
static Put |
makePutFromRegionInfo(RegionInfo regionInfo,
long ts)
Generates and returns a Put containing the region into for the catalog table
|
static Put |
makePutFromTableState(TableState state)
Construct PUT for given state
|
static void |
mergeRegions(Connection connection,
RegionInfo mergedRegion,
RegionInfo regionA,
RegionInfo regionB,
ServerName sn,
int regionReplication,
long masterSystemTime,
boolean saveBarrier)
Merge the two regions into one in an atomic operation.
|
static void |
multiMutate(Connection connection,
Table table,
byte[] row,
List<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 |
overwriteRegions(Connection connection,
List<RegionInfo> regionInfos,
int regionReplication)
Overwrites the specified regions from hbase:meta.
|
static RegionInfo |
parseRegionInfoFromRegionName(byte[] regionName)
Returns an HRI parsed from this regionName.
|
static void |
putsToMetaTable(Connection connection,
List<Put> ps)
Put the passed
ps to the hbase:meta table. |
static void |
putToMetaTable(Connection connection,
Put... puts)
Put the passed
puts 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 |
scanMeta(Connection connection,
byte[] startRow,
byte[] stopRow,
MetaTableAccessor.QueryType type,
int maxRows,
MetaTableAccessor.Visitor visitor)
Performs a scan of META table.
|
static void |
scanMeta(Connection connection,
byte[] startRow,
byte[] stopRow,
MetaTableAccessor.QueryType type,
MetaTableAccessor.Visitor visitor) |
static void |
scanMeta(Connection connection,
MetaTableAccessor.Visitor visitor,
TableName tableName,
byte[] row,
int rowLimit)
Performs a scan of META table for given table starting from
given row.
|
static void |
scanMeta(Connection connection,
TableName table,
MetaTableAccessor.QueryType type,
int maxRows,
MetaTableAccessor.Visitor visitor) |
static void |
scanMetaForTableRegions(Connection connection,
MetaTableAccessor.Visitor visitor,
TableName tableName) |
static void |
splitRegion(Connection connection,
RegionInfo parent,
RegionInfo splitA,
RegionInfo splitB,
ServerName sn,
int regionReplication,
boolean saveBarrier)
Splits the region into two in an atomic operation.
|
static boolean |
tableExists(Connection connection,
TableName tableName)
Checks if the specified table exists.
|
static void |
updateRegionLocation(Connection connection,
RegionInfo 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.
|
static void |
updateReplicationPositions(Connection connection,
String peerId,
Map<String,Long> positions)
Updates the progress of pushing entries to peer cluster.
|
static void |
updateTableState(Connection conn,
TableName tableName,
TableState.State actual)
Updates state in META
|
static void |
updateTableState(Connection connection,
TableState state)
Update state of the table in meta.
|
protected static final char META_REPLICA_ID_DELIMITER
@Deprecated public static NavigableMap<RegionInfo,ServerName> allTableRegions(Connection connection, TableName tableName) throws IOException
getTableRegionsAndLocations(org.apache.hadoop.hbase.client.Connection, org.apache.hadoop.hbase.TableName)
, region can have multiple locationsconnection
- what we will usetableName
- table to listIOException
public static void fullScanRegions(Connection connection, MetaTableAccessor.Visitor visitor) throws IOException
hbase:meta
for regions.connection
- connection we're usingvisitor
- Visitor invoked against each row in regions family.IOException
public static List<Result> fullScanRegions(Connection connection) throws IOException
hbase:meta
for regions.connection
- connection we're usingIOException
public static void fullScanTables(Connection connection, MetaTableAccessor.Visitor visitor) throws IOException
hbase:meta
for tables.connection
- connection we're usingvisitor
- Visitor invoked against each row in tables family.IOException
public static List<Result> fullScan(Connection connection, MetaTableAccessor.QueryType type) throws IOException
hbase:meta
.connection
- connection we're usingtype
- scanned part of metaResult
IOException
public static Table getMetaHTable(Connection connection) throws IOException
Table
instance.connection
- connection we're using to access MetaTable
for hbase:meta
IOException
@Deprecated public static Pair<RegionInfo,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, RegionInfo regionInfo) throws IOException
connection
- connection we're usingregionInfo
- region informationIOException
public static byte[] getMetaKeyForRegion(RegionInfo regionInfo)
public static RegionInfo 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
@Nullable public static Pair<RegionInfo,RegionInfo> 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<RegionInfo> getAllRegions(Connection connection, boolean excludeOfflinedSplitParents) throws IOException
connection
- to connect withexcludeOfflinedSplitParents
- False if we are to include offlined/splitparents regions,
true and we'll leave out offlined regions from returned listIOException
public static List<RegionInfo> getTableRegions(Connection connection, TableName tableName) throws IOException
connection
- connection we're usingtableName
- table we're looking forRegionInfo
.IOException
public static List<RegionInfo> getTableRegions(Connection connection, TableName tableName, boolean excludeOfflinedSplitParents) throws IOException
connection
- connection we're usingtableName
- table we're looking forexcludeOfflinedSplitParents
- If true, do not include offlined split
parents in the return.RegionInfo
.IOException
public static byte[] getTableStartRowForMeta(TableName tableName, MetaTableAccessor.QueryType type)
tableName
- table we're working withpublic static byte[] getTableStopRowForMeta(TableName tableName, MetaTableAccessor.QueryType type)
tableName
- table we're working with@Deprecated public static Scan getScanForTableName(Connection connection, TableName tableName)
tableName
- bytes of table's namepublic static List<Pair<RegionInfo,ServerName>> getTableRegionsAndLocations(Connection connection, TableName tableName) throws IOException
connection
- connection we're usingtableName
- table we're looking forIOException
public static List<Pair<RegionInfo,ServerName>> getTableRegionsAndLocations(Connection connection, @Nullable TableName tableName, boolean excludeOfflinedSplitParents) throws IOException
connection
- connection we're usingtableName
- table to work with, can be null for getting all regionsexcludeOfflinedSplitParents
- don't return split parentsIOException
public static NavigableMap<RegionInfo,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 scanMetaForTableRegions(Connection connection, MetaTableAccessor.Visitor visitor, TableName tableName) throws IOException
IOException
public static void scanMeta(Connection connection, TableName table, MetaTableAccessor.QueryType type, int maxRows, MetaTableAccessor.Visitor visitor) throws IOException
IOException
public static void scanMeta(Connection connection, @Nullable byte[] startRow, @Nullable byte[] stopRow, MetaTableAccessor.QueryType type, MetaTableAccessor.Visitor visitor) throws IOException
IOException
public static void scanMeta(Connection connection, MetaTableAccessor.Visitor visitor, TableName tableName, byte[] row, int rowLimit) throws IOException
connection
- connection we're usingvisitor
- visitor to calltableName
- table withing we scanrow
- start scan from this rowrowLimit
- max number of rows to returnIOException
public static void scanMeta(Connection connection, @Nullable byte[] startRow, @Nullable byte[] stopRow, MetaTableAccessor.QueryType type, int maxRows, MetaTableAccessor.Visitor visitor) throws IOException
connection
- connection we're usingstartRow
- Where to start the scan. Pass null if want to begin scan
at first row.stopRow
- Where to stop the scan. Pass null if want to scan all rows
from the start onetype
- scanned part of metamaxRows
- maximum rows to returnvisitor
- Visitor invoked against each row.IOException
@NonNull public static RegionInfo getClosestRegionInfo(Connection connection, @NonNull TableName tableName, @NonNull byte[] row) throws IOException
row
IOException
protected static byte[] getCatalogFamily()
protected static byte[] getTableFamily()
protected static byte[] getRegionInfoColumn()
protected static byte[] getTableStateColumn()
protected static byte[] getRegionStateColumn()
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 region@Nullable @InterfaceAudience.Private public static ServerName getServerName(Result r, int replicaId)
ServerName
from catalog table Result
.r
- Result to pull from@Nullable public static RegionLocations getRegionLocations(Result r)
public static RegionInfo getRegionInfo(Result data)
data
- a Result object from the catalog table scanpublic static PairOfSameType<RegionInfo> getDaughterRegions(Result data)
data
- a Result object from the catalog table scanpublic static PairOfSameType<RegionInfo> getMergeRegions(Result data)
data
- a Result object from the catalog table scan@Nullable public static TableState getTableState(Connection conn, TableName tableName) throws IOException
conn
- connection to usetableName
- table to fetch state forIOException
public static Map<TableName,TableState> getTableStates(Connection conn) throws IOException
conn
- connection to useIOException
public static void updateTableState(Connection conn, TableName tableName, TableState.State actual) throws IOException
conn
- connection to usetableName
- table to look forIOException
@Nullable public static TableState getTableState(Result r) throws IOException
r
- resultIOException
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(RegionInfo regionInfo) throws IOException
IOException
public static Put makePutFromRegionInfo(RegionInfo regionInfo, long ts) throws IOException
IOException
public static Delete makeDeleteFromRegionInfo(RegionInfo regionInfo)
public static Delete makeDeleteFromRegionInfo(RegionInfo regionInfo, long ts)
public static Put makeBarrierPut(byte[] encodedRegionName, long seq, byte[] tableName) throws IOException
IOException
public static Put makeDaughterPut(byte[] encodedRegionName, byte[] value) throws IOException
IOException
public static Put makeParentPut(byte[] encodedRegionName, byte[] value) throws IOException
IOException
public static Put addDaughtersToPut(Put put, RegionInfo splitA, RegionInfo splitB) throws IOException
IOException
public static void putToMetaTable(Connection connection, Put... puts) throws IOException
puts
to the hbase:meta
table.
Non-atomic for multi puts.connection
- connection we're usingputs
- Put to add to hbase:metaIOException
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
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 addSpiltsToParent(Connection connection, RegionInfo regionInfo, RegionInfo splitA, RegionInfo splitB) throws IOException
splitRegion(Connection, RegionInfo, RegionInfo, RegionInfo, ServerName,int,boolean)
if you want to do that.connection
- connection we're usingregionInfo
- RegionInfo of parent regionsplitA
- 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, RegionInfo regionInfo) throws IOException
connection
- connection we're usingregionInfo
- region informationIOException
- if problem connecting or updating metapublic static void addRegionsToMeta(Connection connection, List<RegionInfo> 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<RegionInfo> 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 mergeRegions(Connection connection, RegionInfo mergedRegion, RegionInfo regionA, RegionInfo regionB, ServerName sn, int regionReplication, long masterSystemTime, boolean saveBarrier) throws IOException
connection
- connection we're usingmergedRegion
- the merged regionregionA
- regionB
- sn
- the location of the regionmasterSystemTime
- saveBarrier
- true if need save replication barrier in meta, used for serial replicationIOException
public static void splitRegion(Connection connection, RegionInfo parent, RegionInfo splitA, RegionInfo splitB, ServerName sn, int regionReplication, boolean saveBarrier) 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 regionsaveBarrier
- true if need save replication barrier in meta, used for serial replicationIOException
public static void updateTableState(Connection connection, TableState state) throws IOException
connection
- what we use for updatestate
- new stateIOException
public static Put makePutFromTableState(TableState state)
state
- new statepublic static void deleteTableState(Connection connection, TableName table) throws IOException
connection
- to use for deletiontable
- to delete state forIOException
public static void multiMutate(Connection connection, Table table, byte[] row, List<Mutation> mutations) throws IOException
IOException
public static void updateRegionLocation(Connection connection, RegionInfo 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
public static void updateReplicationPositions(Connection connection, String peerId, Map<String,Long> positions) throws IOException
connection
- connection we're usingpeerId
- the peerId to pushpositions
- map that saving positions for each regionIOException
public static void deleteRegion(Connection connection, RegionInfo regionInfo) throws IOException
connection
- connection we're usingregionInfo
- region to be deleted from METAIOException
public static void deleteRegions(Connection connection, List<RegionInfo> regionsInfo) throws IOException
connection
- connection we're usingregionsInfo
- list of regions to be deleted from METAIOException
public static void deleteRegions(Connection connection, List<RegionInfo> regionsInfo, long ts) throws IOException
connection
- connection we're usingregionsInfo
- list of regions to be deleted from METAIOException
public static void overwriteRegions(Connection connection, List<RegionInfo> regionInfos, int regionReplication) throws IOException
connection
- connection we're usingregionInfos
- list of regions to be added to METAIOException
public static void deleteMergeQualifiers(Connection connection, RegionInfo mergedRegion) throws IOException
connection
- connection we're usingmergedRegion
- IOException
public static Put addRegionInfo(Put p, RegionInfo hri) throws IOException
IOException
public static Put addLocation(Put p, ServerName sn, long openSeqNum, long time, int replicaId) throws IOException
IOException
public static Put addEmptyLocation(Put p, int replicaId) throws IOException
IOException
public static Put addSequenceNum(Put p, long openSeqNum, long time, int replicaId) throws IOException
IOException
public static long getReplicationPositionForOnePeer(Connection connection, byte[] encodedRegionName, String peerId) throws IOException
connection
- connection we're usingIOException
public static Map<String,Long> getReplicationPositionForAllPeer(Connection connection, byte[] encodedRegionName) throws IOException
connection
- connection we're usingencodedRegionName
- region's encoded nameIOException
public static List<Long> getReplicationBarriers(Connection connection, byte[] encodedRegionName) throws IOException
encodedRegionName
- region's encoded nameIOException
public static Map<String,List<Long>> getAllBarriers(Connection connection) throws IOException
IOException
public static String getSerialReplicationDaughterRegion(Connection connection, byte[] encodedName) throws IOException
connection
- connection we're usingencodedName
- region's encoded nameIOException
public static String getSerialReplicationParentRegion(Connection connection, byte[] encodedName) throws IOException
connection
- connection we're usingencodedName
- region's encoded nameIOException
public static String getSerialReplicationTableName(Connection connection, byte[] encodedName) throws IOException
connection
- connection we're usingencodedName
- region's encoded nameIOException
Copyright © 2007–2018 The Apache Software Foundation. All rights reserved.