@InterfaceAudience.Private public class ZKAssign extends Object
ZKUtil
to cover specific assignment operations.
Contains only static methods and constants.
Used by both the Master and RegionServer.
All valid transitions outlined below:
MASTER
REGIONSERVER
Constructor and Description |
---|
ZKAssign() |
Modifier and Type | Method and Description |
---|---|
static void |
asyncCreateNodeOffline(ZooKeeperWatcher zkw,
HRegionInfo region,
ServerName serverName,
org.apache.zookeeper.AsyncCallback.StringCallback cb,
Object ctx)
Creates an unassigned node in the OFFLINE state for the specified region.
|
static void |
blockUntilNoRIT(ZooKeeperWatcher zkw)
Blocks until there are no node in regions in transition.
|
static void |
blockUntilRIT(ZooKeeperWatcher zkw)
Blocks until there is at least one node in regions in transition.
|
static boolean |
checkClosingState(ZooKeeperWatcher zkw,
HRegionInfo region,
int expectedVersion) |
static int |
confirmNodeOpening(ZooKeeperWatcher zkw,
HRegionInfo region,
ServerName serverName,
int expectedVersion)
Confirm an existing unassigned node for the specified region which is
currently in the OPENING state to be still in the OPENING state on
the specified server.
|
static int |
createNodeClosing(ZooKeeperWatcher zkw,
HRegionInfo region,
ServerName serverName)
Creates a new unassigned node in the CLOSING state for the specified
region.
|
static void |
createNodeOffline(ZooKeeperWatcher zkw,
HRegionInfo region,
ServerName serverName)
Creates a new unassigned node in the OFFLINE state for the specified region.
|
static void |
createNodeOffline(ZooKeeperWatcher zkw,
HRegionInfo region,
ServerName serverName,
EventType event) |
static int |
createOrForceNodeOffline(ZooKeeperWatcher zkw,
HRegionInfo region,
ServerName serverName)
Creates or force updates an unassigned node to the OFFLINE state for the
specified region.
|
static void |
deleteAllNodes(ZooKeeperWatcher zkw)
Deletes all unassigned nodes regardless of their state.
|
static boolean |
deleteClosedNode(ZooKeeperWatcher zkw,
String encodedRegionName,
ServerName sn)
Deletes an existing unassigned node that is in the CLOSED state for the
specified region.
|
static boolean |
deleteClosingNode(ZooKeeperWatcher zkw,
HRegionInfo region,
ServerName sn)
Deletes an existing unassigned node that is in the CLOSING state for the
specified region.
|
static boolean |
deleteNode(ZooKeeperWatcher zkw,
String encodedRegionName,
EventType expectedState,
int expectedVersion)
Deletes an existing unassigned node that is in the specified state for the
specified region.
|
static boolean |
deleteNode(ZooKeeperWatcher zkw,
String encodedRegionName,
EventType expectedState,
ServerName sn)
Deletes an existing unassigned node that is in the specified state for the
specified region.
|
static boolean |
deleteNode(ZooKeeperWatcher zkw,
String encodedRegionName,
EventType expectedState,
ServerName serverName,
int expectedVersion)
Deletes an existing unassigned node that is in the specified state for the
specified region.
|
static void |
deleteNodeFailSilent(ZooKeeperWatcher watcher,
HRegionInfo regionInfo)
Delete the assignment node regardless of its current state.
|
static boolean |
deleteOfflineNode(ZooKeeperWatcher zkw,
String encodedRegionName,
ServerName sn)
Deletes an existing unassigned node that is in the OFFLINE state for the
specified region.
|
static boolean |
deleteOpenedNode(ZooKeeperWatcher zkw,
String encodedRegionName,
ServerName sn)
Deletes an existing unassigned node that is in the OPENED state for the
specified region.
|
static byte[] |
getData(ZooKeeperWatcher zkw,
String pathOrRegionName)
Gets the current data in the unassigned node for the specified region name
or fully-qualified path.
|
static byte[] |
getDataAndWatch(ZooKeeperWatcher zkw,
String pathOrRegionName,
org.apache.zookeeper.data.Stat stat)
Gets the current data in the unassigned node for the specified region name
or fully-qualified path.
|
static byte[] |
getDataNoWatch(ZooKeeperWatcher zkw,
String pathOrRegionName,
org.apache.zookeeper.data.Stat stat)
Gets the current data in the unassigned node for the specified region name
or fully-qualified path.
|
static String |
getNodeName(ZooKeeperWatcher zkw,
String regionName)
Gets the full path node name for the unassigned node for the specified
region.
|
static String |
getPath(ZooKeeperWatcher zkw,
String pathOrRegionName) |
static String |
getRegionName(ZooKeeperWatcher zkw,
String path)
Gets the region name from the full path node name of an unassigned node.
|
static int |
getVersion(ZooKeeperWatcher zkw,
HRegionInfo region)
Get the version of the specified znode
|
static int |
transitionNode(ZooKeeperWatcher zkw,
HRegionInfo region,
ServerName serverName,
EventType beginState,
EventType endState,
int expectedVersion)
Method that actually performs unassigned node transitions.
|
static int |
transitionNode(ZooKeeperWatcher zkw,
HRegionInfo region,
ServerName serverName,
EventType beginState,
EventType endState,
int expectedVersion,
byte[] payload) |
static int |
transitionNodeClosed(ZooKeeperWatcher zkw,
HRegionInfo region,
ServerName serverName,
int expectedVersion)
Transitions an existing unassigned node for the specified region which is
currently in the CLOSING state to be in the CLOSED state.
|
static int |
transitionNodeOpened(ZooKeeperWatcher zkw,
HRegionInfo region,
ServerName serverName,
int expectedVersion)
Transitions an existing unassigned node for the specified region which is
currently in the OPENING state to be in the OPENED state.
|
static int |
transitionNodeOpening(ZooKeeperWatcher zkw,
HRegionInfo region,
ServerName serverName)
Transitions an existing unassigned node for the specified region which is
currently in the OFFLINE state to be in the OPENING state.
|
static int |
transitionNodeOpening(ZooKeeperWatcher zkw,
HRegionInfo region,
ServerName serverName,
EventType beginState) |
public static String getNodeName(ZooKeeperWatcher zkw, String regionName)
zkw
- zk referenceregionName
- region namepublic static String getRegionName(ZooKeeperWatcher zkw, String path)
path
- full zk pathpublic static void createNodeOffline(ZooKeeperWatcher zkw, HRegionInfo region, ServerName serverName) throws org.apache.zookeeper.KeeperException, org.apache.zookeeper.KeeperException.NodeExistsException
Does not transition nodes from other states. If a node already exists
for this region, a KeeperException.NodeExistsException
will be thrown.
Sets a watcher on the unassigned region node if the method is successful.
This method should only be used during cluster startup and the enabling of a table.
zkw
- zk referenceregion
- region to be created as offlineserverName
- server transition will happen onorg.apache.zookeeper.KeeperException
- if unexpected zookeeper exceptionKeeperException.NodeExistsException
- if node already existspublic static void createNodeOffline(ZooKeeperWatcher zkw, HRegionInfo region, ServerName serverName, EventType event) throws org.apache.zookeeper.KeeperException, org.apache.zookeeper.KeeperException.NodeExistsException
org.apache.zookeeper.KeeperException
org.apache.zookeeper.KeeperException.NodeExistsException
public static void asyncCreateNodeOffline(ZooKeeperWatcher zkw, HRegionInfo region, ServerName serverName, org.apache.zookeeper.AsyncCallback.StringCallback cb, Object ctx) throws org.apache.zookeeper.KeeperException
Runs asynchronously. Depends on no pre-existing znode.
Sets a watcher on the unassigned region node.
zkw
- zk referenceregion
- region to be created as offlineserverName
- server transition will happen oncb
- ctx
- org.apache.zookeeper.KeeperException
- if unexpected zookeeper exceptionKeeperException.NodeExistsException
- if node already existspublic static int createOrForceNodeOffline(ZooKeeperWatcher zkw, HRegionInfo region, ServerName serverName) throws org.apache.zookeeper.KeeperException
Attempts to create the node but if it exists will force it to transition to and OFFLINE state.
Sets a watcher on the unassigned region node if the method is successful.
This method should be used when assigning a region.
zkw
- zk referenceregion
- region to be created as offlineserverName
- server transition will happen onorg.apache.zookeeper.KeeperException
- if unexpected zookeeper exceptionKeeperException.NodeExistsException
- if node already existspublic static boolean deleteOpenedNode(ZooKeeperWatcher zkw, String encodedRegionName, ServerName sn) throws org.apache.zookeeper.KeeperException, org.apache.zookeeper.KeeperException.NoNodeException
If a node does not already exist for this region, a
KeeperException.NoNodeException
will be thrown.
No watcher is set whether this succeeds or not.
Returns false if the node was not in the proper state but did exist.
This method is used during normal region transitions when a region finishes successfully opening. This is the Master acknowledging completion of the specified regions transition.
zkw
- zk referenceencodedRegionName
- opened region to be deleted from zksn
- the expected region transition target server nameorg.apache.zookeeper.KeeperException
- if unexpected zookeeper exceptionKeeperException.NoNodeException
- if node does not existpublic static boolean deleteOfflineNode(ZooKeeperWatcher zkw, String encodedRegionName, ServerName sn) throws org.apache.zookeeper.KeeperException, org.apache.zookeeper.KeeperException.NoNodeException
If a node does not already exist for this region, a
KeeperException.NoNodeException
will be thrown.
No watcher is set whether this succeeds or not.
Returns false if the node was not in the proper state but did exist.
This method is used during master failover when the regions on an RS that has died are all set to OFFLINE before being processed.
zkw
- zk referenceencodedRegionName
- closed region to be deleted from zksn
- the expected region transition target server nameorg.apache.zookeeper.KeeperException
- if unexpected zookeeper exceptionKeeperException.NoNodeException
- if node does not existpublic static boolean deleteClosedNode(ZooKeeperWatcher zkw, String encodedRegionName, ServerName sn) throws org.apache.zookeeper.KeeperException, org.apache.zookeeper.KeeperException.NoNodeException
If a node does not already exist for this region, a
KeeperException.NoNodeException
will be thrown.
No watcher is set whether this succeeds or not.
Returns false if the node was not in the proper state but did exist.
This method is used during table disables when a region finishes successfully closing. This is the Master acknowledging completion of the specified regions transition to being closed.
zkw
- zk referenceencodedRegionName
- closed region to be deleted from zksn
- the expected region transition target server nameorg.apache.zookeeper.KeeperException
- if unexpected zookeeper exceptionKeeperException.NoNodeException
- if node does not existpublic static boolean deleteClosingNode(ZooKeeperWatcher zkw, HRegionInfo region, ServerName sn) throws org.apache.zookeeper.KeeperException, org.apache.zookeeper.KeeperException.NoNodeException
If a node does not already exist for this region, a
KeeperException.NoNodeException
will be thrown.
No watcher is set whether this succeeds or not.
Returns false if the node was not in the proper state but did exist.
This method is used during table disables when a region finishes successfully closing. This is the Master acknowledging completion of the specified regions transition to being closed.
zkw
- zk referenceregion
- closing region to be deleted from zksn
- the expected region transition target server nameorg.apache.zookeeper.KeeperException
- if unexpected zookeeper exceptionKeeperException.NoNodeException
- if node does not existpublic static boolean deleteNode(ZooKeeperWatcher zkw, String encodedRegionName, EventType expectedState, ServerName sn) throws org.apache.zookeeper.KeeperException, org.apache.zookeeper.KeeperException.NoNodeException
If a node does not already exist for this region, a
KeeperException.NoNodeException
will be thrown.
No watcher is set whether this succeeds or not.
Returns false if the node was not in the proper state but did exist.
This method is used when a region finishes opening/closing. The Master acknowledges completion of the specified regions transition to being closed/opened.
zkw
- zk referenceencodedRegionName
- region to be deleted from zkexpectedState
- state region must be in for delete to completesn
- the expected region transition target server nameorg.apache.zookeeper.KeeperException
- if unexpected zookeeper exceptionKeeperException.NoNodeException
- if node does not existpublic static boolean deleteNode(ZooKeeperWatcher zkw, String encodedRegionName, EventType expectedState, int expectedVersion) throws org.apache.zookeeper.KeeperException, org.apache.zookeeper.KeeperException.NoNodeException
If a node does not already exist for this region, a
KeeperException.NoNodeException
will be thrown.
No watcher is set whether this succeeds or not.
Returns false if the node was not in the proper state but did exist.
This method is used when a region finishes opening/closing. The Master acknowledges completion of the specified regions transition to being closed/opened.
zkw
- zk referenceencodedRegionName
- region to be deleted from zkexpectedState
- state region must be in for delete to completeexpectedVersion
- of the znode that is to be deleted.
If expectedVersion need not be compared while deleting the znode
pass -1org.apache.zookeeper.KeeperException
- if unexpected zookeeper exceptionKeeperException.NoNodeException
- if node does not existpublic static boolean deleteNode(ZooKeeperWatcher zkw, String encodedRegionName, EventType expectedState, ServerName serverName, int expectedVersion) throws org.apache.zookeeper.KeeperException, org.apache.zookeeper.KeeperException.NoNodeException
If a node does not already exist for this region, a
KeeperException.NoNodeException
will be thrown.
No watcher is set whether this succeeds or not.
Returns false if the node was not in the proper state but did exist.
This method is used when a region finishes opening/closing. The Master acknowledges completion of the specified regions transition to being closed/opened.
zkw
- zk referenceencodedRegionName
- region to be deleted from zkexpectedState
- state region must be in for delete to completeserverName
- the expected region transition target server nameexpectedVersion
- of the znode that is to be deleted.
If expectedVersion need not be compared while deleting the znode
pass -1org.apache.zookeeper.KeeperException
- if unexpected zookeeper exceptionKeeperException.NoNodeException
- if node does not existpublic static void deleteAllNodes(ZooKeeperWatcher zkw) throws org.apache.zookeeper.KeeperException
No watchers are set.
This method is used by the Master during cluster startup to clear out any existing state from other cluster runs.
zkw
- zk referenceorg.apache.zookeeper.KeeperException
- if unexpected zookeeper exceptionpublic static int createNodeClosing(ZooKeeperWatcher zkw, HRegionInfo region, ServerName serverName) throws org.apache.zookeeper.KeeperException, org.apache.zookeeper.KeeperException.NodeExistsException
Does not transition nodes from any states. If a node already exists
for this region, a KeeperException.NodeExistsException
will be thrown.
If creation is successful, returns the version number of the CLOSING node created.
Set a watch.
This method should only be used by a Master when initiating a close of a region before sending a close request to the region server.
zkw
- zk referenceregion
- region to be created as closingserverName
- server transition will happen onorg.apache.zookeeper.KeeperException
- if unexpected zookeeper exceptionKeeperException.NodeExistsException
- if node already existspublic static int transitionNodeClosed(ZooKeeperWatcher zkw, HRegionInfo region, ServerName serverName, int expectedVersion) throws org.apache.zookeeper.KeeperException
Does not transition nodes from other states. If for some reason the node could not be transitioned, the method returns -1. If the transition is successful, the version of the node after transition is returned.
This method can fail and return false for three different reasons:
Does not set any watches.
This method should only be used by a RegionServer when initiating a close of a region after receiving a CLOSE RPC from the Master.
zkw
- zk referenceregion
- region to be transitioned to closedserverName
- server transition happens onorg.apache.zookeeper.KeeperException
- if unexpected zookeeper exceptionpublic static int transitionNodeOpening(ZooKeeperWatcher zkw, HRegionInfo region, ServerName serverName) throws org.apache.zookeeper.KeeperException
Does not transition nodes from other states. If for some reason the node could not be transitioned, the method returns -1. If the transition is successful, the version of the node written as OPENING is returned.
This method can fail and return -1 for three different reasons:
Does not set any watches.
This method should only be used by a RegionServer when initiating an open of a region after receiving an OPEN RPC from the Master.
zkw
- zk referenceregion
- region to be transitioned to openingserverName
- server transition happens onorg.apache.zookeeper.KeeperException
- if unexpected zookeeper exceptionpublic static int transitionNodeOpening(ZooKeeperWatcher zkw, HRegionInfo region, ServerName serverName, EventType beginState) throws org.apache.zookeeper.KeeperException
org.apache.zookeeper.KeeperException
public static int confirmNodeOpening(ZooKeeperWatcher zkw, HRegionInfo region, ServerName serverName, int expectedVersion) throws org.apache.zookeeper.KeeperException
If for some reason the check fails, the method returns -1. Otherwise, the version of the node (same as the expected version) is returned.
This method can fail and return -1 for three different reasons:
Does not set any watches.
This method should only be used by a RegionServer when initiating an open of a region after receiving an OPEN RPC from the Master.
zkw
- zk referenceregion
- region to be transitioned to openingserverName
- server transition happens onorg.apache.zookeeper.KeeperException
- if unexpected zookeeper exceptionpublic static int transitionNodeOpened(ZooKeeperWatcher zkw, HRegionInfo region, ServerName serverName, int expectedVersion) throws org.apache.zookeeper.KeeperException
Does not transition nodes from other states. If for some reason the node could not be transitioned, the method returns -1. If the transition is successful, the version of the node after transition is returned.
This method can fail and return false for three different reasons:
Does not set any watches.
This method should only be used by a RegionServer when completing the open of a region.
zkw
- zk referenceregion
- region to be transitioned to openedserverName
- server transition happens onorg.apache.zookeeper.KeeperException
- if unexpected zookeeper exceptionpublic static boolean checkClosingState(ZooKeeperWatcher zkw, HRegionInfo region, int expectedVersion) throws org.apache.zookeeper.KeeperException
zkw
- zk referenceregion
- region to be closedexpectedVersion
- expected version of the znodeorg.apache.zookeeper.KeeperException
public static int transitionNode(ZooKeeperWatcher zkw, HRegionInfo region, ServerName serverName, EventType beginState, EventType endState, int expectedVersion) throws org.apache.zookeeper.KeeperException
Attempts to transition the unassigned node for the specified region from the expected state to the state in the specified transition data.
Method first reads existing data and verifies it is in the expected state. If the node does not exist or the node is not in the expected state, the method returns -1. If the transition is successful, the version number of the node following the transition is returned.
If the read state is what is expected, it attempts to write the new state and data into the node. When doing this, it includes the expected version (determined when the existing state was verified) to ensure that only one transition is successful. If there is a version mismatch, the method returns -1.
If the write is successful, no watch is set and the method returns true.
zkw
- zk referenceregion
- region to be transitioned to openedserverName
- server transition happens onendState
- state to transition node to if all checks passbeginState
- state the node must currently be in to do transitionexpectedVersion
- expected version of data before modification, or -1org.apache.zookeeper.KeeperException
- if unexpected zookeeper exceptionpublic static int transitionNode(ZooKeeperWatcher zkw, HRegionInfo region, ServerName serverName, EventType beginState, EventType endState, int expectedVersion, byte[] payload) throws org.apache.zookeeper.KeeperException
org.apache.zookeeper.KeeperException
public static byte[] getData(ZooKeeperWatcher zkw, String pathOrRegionName) throws org.apache.zookeeper.KeeperException
Returns null if the region does not currently have a node.
Sets a watch on the node if the node exists.
zkw
- zk referencepathOrRegionName
- fully-specified path or region nameorg.apache.zookeeper.KeeperException
- if unexpected zookeeper exceptionpublic static byte[] getDataAndWatch(ZooKeeperWatcher zkw, String pathOrRegionName, org.apache.zookeeper.data.Stat stat) throws org.apache.zookeeper.KeeperException
Returns null if the region does not currently have a node.
Sets a watch on the node if the node exists.
zkw
- zk referencepathOrRegionName
- fully-specified path or region namestat
- object to populate the version.org.apache.zookeeper.KeeperException
- if unexpected zookeeper exceptionpublic static byte[] getDataNoWatch(ZooKeeperWatcher zkw, String pathOrRegionName, org.apache.zookeeper.data.Stat stat) throws org.apache.zookeeper.KeeperException
Returns null if the region does not currently have a node.
Does not set a watch.
zkw
- zk referencepathOrRegionName
- fully-specified path or region namestat
- object to store node info into on getData callorg.apache.zookeeper.KeeperException
- if unexpected zookeeper exceptionpublic static String getPath(ZooKeeperWatcher zkw, String pathOrRegionName)
zkw
- pathOrRegionName
- public static int getVersion(ZooKeeperWatcher zkw, HRegionInfo region) throws org.apache.zookeeper.KeeperException
zkw
- zk referenceregion
- region's infoorg.apache.zookeeper.KeeperException
public static void deleteNodeFailSilent(ZooKeeperWatcher watcher, HRegionInfo regionInfo) throws org.apache.zookeeper.KeeperException
Fail silent even if the node does not exist at all.
watcher
- regionInfo
- org.apache.zookeeper.KeeperException
public static void blockUntilNoRIT(ZooKeeperWatcher zkw) throws org.apache.zookeeper.KeeperException, InterruptedException
Used in testing only.
zkw
- zk referenceorg.apache.zookeeper.KeeperException
InterruptedException
public static void blockUntilRIT(ZooKeeperWatcher zkw) throws org.apache.zookeeper.KeeperException, InterruptedException
Used in testing only.
zkw
- zk referenceorg.apache.zookeeper.KeeperException
InterruptedException
Copyright © 2007-2016 The Apache Software Foundation. All Rights Reserved.