Class BalancerClusterState
java.lang.Object
org.apache.hadoop.hbase.master.balancer.BalancerClusterState
An efficient array based implementation similar to ClusterState for keeping the status of the
cluster in terms of region assignment and distribution. LoadBalancers, such as
StochasticLoadBalancer uses this Cluster object because of hundreds of thousands of hashmap
manipulations are very costly, which is why this class uses mostly indexes and arrays.
BalancerClusterState tracks a list of unassigned regions, region assignments, and the server
topology in terms of server names, hostnames and racks.
-
Nested Class Summary
Modifier and TypeClassDescription(package private) static class
(package private) static enum
-
Field Summary
Modifier and TypeFieldDescription(package private) Map<ServerName,
List<RegionInfo>> (package private) org.agrona.collections.Int2IntCounterMap[]
(package private) org.agrona.collections.Int2IntCounterMap[]
(package private) org.agrona.collections.Int2IntCounterMap[]
(package private) boolean
(package private) String[]
(package private) int[]
(package private) float[]
private static final org.slf4j.Logger
(package private) double[]
(package private) boolean
(package private) int
(package private) int
(package private) int
(package private) int
private Comparator<Integer>
(package private) int[][]
(package private) int[]
(package private) int
(package private) int
private float[][]
private final RackManager
(package private) String[]
(package private) Map<String,
Pair<ServerName, Float>> private RegionHDFSBlockLocationFinder
(package private) int[]
(package private) int[]
(package private) int[]
(package private) Deque<BalancerRegionLoad>[]
(package private) int[][]
(package private) RegionInfo[]
private int[]
(package private) int[][]
(package private) int[][]
(package private) int[][]
(package private) Map<RegionInfo,
Integer> private int[][]
(package private) int[]
(package private) int[]
(package private) int[]
(package private) Integer[]
(package private) Integer[]
(package private) ServerName[]
(package private) int[][]
(package private) int[][]
-
Constructor Summary
ModifierConstructorDescription(package private)
BalancerClusterState
(Collection<RegionInfo> unassignedRegions, Map<ServerName, List<RegionInfo>> clusterState, Map<String, Deque<BalancerRegionLoad>> loads, RegionHDFSBlockLocationFinder regionFinder, RackManager rackManager, Map<String, Pair<ServerName, Float>> oldRegionServerRegionCacheRatio) (package private)
BalancerClusterState
(Map<ServerName, List<RegionInfo>> clusterState, Map<String, Deque<BalancerRegionLoad>> loads, RegionHDFSBlockLocationFinder regionFinder, RackManager rackManager) protected
BalancerClusterState
(Map<ServerName, List<RegionInfo>> clusterState, Map<String, Deque<BalancerRegionLoad>> loads, RegionHDFSBlockLocationFinder regionFinder, RackManager rackManager, Map<String, Pair<ServerName, Float>> oldRegionServerRegionCacheRatio) -
Method Summary
Modifier and TypeMethodDescription(package private) int[]
addRegion
(int[] regions, int regionIndex) (package private) int[]
addRegionSorted
(int[] regions, int regionIndex) private int
checkLocationForPrimary
(int location, org.agrona.collections.Int2IntCounterMap[] colocatedReplicaCountsPerLocation, int primary) Common method for better solution check.private void
Computes and caches the locality for each region/rack combinations, as well as storing a mapping of region -> server and region -> rack such that server and rack have the highest locality for regionprivate void
Populate the maps containing information about how much a region is cached on a region server.(package private) boolean
contains
(int[] arr, int val) void
doAction
(BalanceAction action) (package private) void
doAssignRegion
(RegionInfo regionInfo, ServerName serverName) (package private) float
getLocalityOfRegion
(int region, int server) (package private) int
getLowestLocalityRegionOnServer
(int serverIndex) (package private) int
getNumRegions
(int server) float
getOrComputeLocality
(int region, int entity, BalancerClusterState.LocalityType type) Looks up locality from cache of localities.float[][]
Retrieves and lazily initializes a field storing the locality of every region/server combinationprotected float
getOrComputeRegionCacheRatio
(int region, int server) int[]
Lazily initializes and retrieves a mapping of region -> server for which region has the highest the localityint[]
double
getOrComputeWeightedLocality
(int region, int server, BalancerClusterState.LocalityType type) Returns locality weighted by region size in MB.float
getOrComputeWeightedRegionCacheRatio
(int region, int server) Returns the weighted cache ratio of a region on the given region serverint
getRackForRegion
(int region) Maps region index to rack indexprotected float
getRegionCacheRatioOnRegionServer
(int region, int regionServerIndex) Returns the amount by which a region is cached on a given region server.int
getRegionSizeMB
(int region) Returns the size in MB from the most recent RegionLoad for regionint
getTotalRegionHFileSizeMB
(int region) Returns the size of hFiles from the most recent RegionLoad for regionprivate void
populateRegionPerLocationFromServer
(int[][] regionsPerLocation, org.agrona.collections.Int2IntCounterMap[] colocatedReplicaCountsPerLocation, int[][] serversPerLocation) (package private) void
regionMoved
(int region, int oldServer, int newServer) private void
registerRegion
(RegionInfo region, int regionIndex, int serverIndex, Map<String, Deque<BalancerRegionLoad>> loads, RegionHDFSBlockLocationFinder regionFinder) Helper for Cluster constructor to handle a region(package private) int[]
removeRegion
(int[] regions, int regionIndex) (package private) int[]
replaceRegion
(int[] regions, int regionIndex, int newRegionIndex) boolean
serverHasTooFewRegions
(int server) Returns true iff a given server has less regions than the balanced amount(package private) void
setNumMovedRegions
(int numMovedRegions) (package private) void
setNumRegions
(int numRegions) (package private) void
toString()
private void
updateForLocation
(int[] serverIndexToLocation, int[][] regionsPerLocation, org.agrona.collections.Int2IntCounterMap[] colocatedReplicaCountsPerLocation, int oldServer, int newServer, int primary, int region) Common method for per host and per Location region index updates when a region is moved.(package private) boolean
wouldLowerAvailability
(RegionInfo regionInfo, ServerName serverName) Return true if the placement of region on server would lower the availability of the region in question
-
Field Details
-
LOG
-
servers
-
hosts
-
racks
-
multiServersPerHost
boolean multiServersPerHost -
tables
-
regions
-
regionLoads
-
regionFinder
-
regionLocations
int[][] regionLocations -
serverIndexToHostIndex
int[] serverIndexToHostIndex -
serverIndexToRackIndex
int[] serverIndexToRackIndex -
regionsPerServer
int[][] regionsPerServer -
serverIndexToRegionsOffset
-
regionsPerHost
int[][] regionsPerHost -
regionsPerRack
int[][] regionsPerRack -
colocatedReplicaCountsPerServer
org.agrona.collections.Int2IntCounterMap[] colocatedReplicaCountsPerServer -
colocatedReplicaCountsPerHost
org.agrona.collections.Int2IntCounterMap[] colocatedReplicaCountsPerHost -
colocatedReplicaCountsPerRack
org.agrona.collections.Int2IntCounterMap[] colocatedReplicaCountsPerRack -
serversPerHost
int[][] serversPerHost -
serversPerRack
int[][] serversPerRack -
regionIndexToServerIndex
int[] regionIndexToServerIndex -
initialRegionIndexToServerIndex
-
regionIndexToTableIndex
int[] regionIndexToTableIndex -
numRegionsPerServerPerTable
int[][] numRegionsPerServerPerTable -
numRegionsPerTable
int[] numRegionsPerTable -
meanRegionsPerTable
double[] meanRegionsPerTable -
regionIndexToPrimaryIndex
-
hasRegionReplicas
boolean hasRegionReplicas -
serverIndicesSortedByRegionCount
-
serverIndicesSortedByLocality
-
serversToIndex
-
hostsToIndex
-
racksToIndex
-
tablesToIndex
-
regionsToIndex
-
localityPerServer
float[] localityPerServer -
numServers
int numServers -
numHosts
int numHosts -
numRacks
int numRacks -
numTables
int numTables -
numRegions
int numRegions -
numMovedRegions
int numMovedRegions -
clusterState
-
rackManager
-
rackLocalities
-
regionsToMostLocalEntities
-
regionIndexServerIndexRegionCachedRatio
-
regionServerIndexWithBestRegionCachedRatio
-
regionCacheRatioOnOldServerMap
-
numRegionsComparator
-
-
Constructor Details
-
BalancerClusterState
BalancerClusterState(Map<ServerName, List<RegionInfo>> clusterState, Map<String, Deque<BalancerRegionLoad>> loads, RegionHDFSBlockLocationFinder regionFinder, RackManager rackManager) -
BalancerClusterState
protected BalancerClusterState(Map<ServerName, List<RegionInfo>> clusterState, Map<String, Deque<BalancerRegionLoad>> loads, RegionHDFSBlockLocationFinder regionFinder, RackManager rackManager, Map<String, Pair<ServerName, Float>> oldRegionServerRegionCacheRatio) -
BalancerClusterState
BalancerClusterState(Collection<RegionInfo> unassignedRegions, Map<ServerName, List<RegionInfo>> clusterState, Map<String, Deque<BalancerRegionLoad>> loads, RegionHDFSBlockLocationFinder regionFinder, RackManager rackManager, Map<String, Pair<ServerName, Float>> oldRegionServerRegionCacheRatio)
-
-
Method Details
-
populateRegionPerLocationFromServer
private void populateRegionPerLocationFromServer(int[][] regionsPerLocation, org.agrona.collections.Int2IntCounterMap[] colocatedReplicaCountsPerLocation, int[][] serversPerLocation) -
registerRegion
private void registerRegion(RegionInfo region, int regionIndex, int serverIndex, Map<String, Deque<BalancerRegionLoad>> loads, RegionHDFSBlockLocationFinder regionFinder) Helper for Cluster constructor to handle a region -
serverHasTooFewRegions
Returns true iff a given server has less regions than the balanced amount -
getOrComputeRackLocalities
Retrieves and lazily initializes a field storing the locality of every region/server combination -
getOrComputeRegionsToMostLocalEntities
Lazily initializes and retrieves a mapping of region -> server for which region has the highest the locality -
getOrComputeLocality
Looks up locality from cache of localities. Will create cache if it does not already exist. -
getOrComputeWeightedLocality
public double getOrComputeWeightedLocality(int region, int server, BalancerClusterState.LocalityType type) Returns locality weighted by region size in MB. Will create locality cache if it does not already exist. -
getRegionSizeMB
Returns the size in MB from the most recent RegionLoad for region -
computeCachedLocalities
Computes and caches the locality for each region/rack combinations, as well as storing a mapping of region -> server and region -> rack such that server and rack have the highest locality for region -
getTotalRegionHFileSizeMB
Returns the size of hFiles from the most recent RegionLoad for region -
getOrComputeWeightedRegionCacheRatio
Returns the weighted cache ratio of a region on the given region server -
getRegionCacheRatioOnRegionServer
Returns the amount by which a region is cached on a given region server. If the region is not currently hosted on the given region server, then find out if it was previously hosted there and return the old cache ratio. -
computeRegionServerRegionCacheRatio
Populate the maps containing information about how much a region is cached on a region server. -
getOrComputeRegionCacheRatio
-
getOrComputeServerWithBestRegionCachedRatio
-
getRackForRegion
Maps region index to rack index -
doAction
-
wouldLowerAvailability
Return true if the placement of region on server would lower the availability of the region in question- Returns:
- true or false
-
checkLocationForPrimary
private int checkLocationForPrimary(int location, org.agrona.collections.Int2IntCounterMap[] colocatedReplicaCountsPerLocation, int primary) Common method for better solution check.- Parameters:
colocatedReplicaCountsPerLocation
- colocatedReplicaCountsPerHost or colocatedReplicaCountsPerRack- Returns:
- 1 for better, -1 for no better, 0 for unknown
-
doAssignRegion
-
regionMoved
-
updateForLocation
private void updateForLocation(int[] serverIndexToLocation, int[][] regionsPerLocation, org.agrona.collections.Int2IntCounterMap[] colocatedReplicaCountsPerLocation, int oldServer, int newServer, int primary, int region) Common method for per host and per Location region index updates when a region is moved.- Parameters:
serverIndexToLocation
- serverIndexToHostIndex or serverIndexToLocationIndexregionsPerLocation
- regionsPerHost or regionsPerLocationcolocatedReplicaCountsPerLocation
- colocatedReplicaCountsPerHost or colocatedReplicaCountsPerRack
-
removeRegion
-
addRegion
-
addRegionSorted
-
replaceRegion
-
sortServersByRegionCount
void sortServersByRegionCount() -
getNumRegions
-
contains
-
getNumRegionsComparator
-
getLowestLocalityRegionOnServer
-
getLocalityOfRegion
-
setNumRegions
-
setNumMovedRegions
-
toString
-