|
@@ -23,7 +23,9 @@ import java.io.ByteArrayOutputStream;
|
|
|
import java.io.DataInputStream;
|
|
|
import java.io.DataOutputStream;
|
|
|
import java.io.IOException;
|
|
|
+import java.security.NoSuchAlgorithmException;
|
|
|
import java.util.ArrayList;
|
|
|
+import java.util.Collections;
|
|
|
import java.util.List;
|
|
|
|
|
|
import org.apache.commons.logging.Log;
|
|
@@ -31,6 +33,7 @@ import org.apache.commons.logging.LogFactory;
|
|
|
import org.apache.hadoop.classification.InterfaceAudience.Private;
|
|
|
import org.apache.hadoop.classification.InterfaceStability.Unstable;
|
|
|
import org.apache.hadoop.conf.Configuration;
|
|
|
+import org.apache.hadoop.ha.HAServiceProtocol;
|
|
|
import org.apache.hadoop.io.DataInputByteBuffer;
|
|
|
import org.apache.hadoop.security.Credentials;
|
|
|
import org.apache.hadoop.security.token.delegation.DelegationKey;
|
|
@@ -38,11 +41,14 @@ import org.apache.hadoop.util.StringUtils;
|
|
|
import org.apache.hadoop.util.ZKUtil;
|
|
|
import org.apache.hadoop.yarn.api.records.ApplicationAttemptId;
|
|
|
import org.apache.hadoop.yarn.api.records.ApplicationId;
|
|
|
+import org.apache.hadoop.yarn.client.RMHAServiceTarget;
|
|
|
+import org.apache.hadoop.yarn.conf.HAUtil;
|
|
|
import org.apache.hadoop.yarn.conf.YarnConfiguration;
|
|
|
import org.apache.hadoop.yarn.exceptions.YarnRuntimeException;
|
|
|
import org.apache.hadoop.yarn.proto.YarnServerResourceManagerServiceProtos.ApplicationAttemptStateDataProto;
|
|
|
import org.apache.hadoop.yarn.proto.YarnServerResourceManagerServiceProtos.ApplicationStateDataProto;
|
|
|
import org.apache.hadoop.yarn.security.client.RMDelegationTokenIdentifier;
|
|
|
+import org.apache.hadoop.yarn.server.resourcemanager.ResourceManager;
|
|
|
import org.apache.hadoop.yarn.server.resourcemanager.recovery.records.impl.pb.ApplicationAttemptStateDataPBImpl;
|
|
|
import org.apache.hadoop.yarn.server.resourcemanager.recovery.records.impl.pb.ApplicationStateDataPBImpl;
|
|
|
import org.apache.hadoop.yarn.util.ConverterUtils;
|
|
@@ -53,11 +59,14 @@ import org.apache.zookeeper.Op;
|
|
|
import org.apache.zookeeper.WatchedEvent;
|
|
|
import org.apache.zookeeper.Watcher;
|
|
|
import org.apache.zookeeper.Watcher.Event;
|
|
|
+import org.apache.zookeeper.ZooDefs;
|
|
|
import org.apache.zookeeper.ZooKeeper;
|
|
|
import org.apache.zookeeper.data.ACL;
|
|
|
+import org.apache.zookeeper.data.Id;
|
|
|
import org.apache.zookeeper.data.Stat;
|
|
|
|
|
|
import com.google.common.annotations.VisibleForTesting;
|
|
|
+import org.apache.zookeeper.server.auth.DigestAuthenticationProvider;
|
|
|
|
|
|
@Private
|
|
|
@Unstable
|
|
@@ -83,6 +92,55 @@ public class ZKRMStateStore extends RMStateStore {
|
|
|
protected ZooKeeper zkClient;
|
|
|
private ZooKeeper oldZkClient;
|
|
|
|
|
|
+ /** Fencing related variables */
|
|
|
+ private static final String FENCING_LOCK = "RM_ZK_FENCING_LOCK";
|
|
|
+ private String fencingNodePath;
|
|
|
+ private Op createFencingNodePathOp;
|
|
|
+ private Op deleteFencingNodePathOp;
|
|
|
+
|
|
|
+ @VisibleForTesting
|
|
|
+ List<ACL> zkRootNodeAcl;
|
|
|
+ private boolean useDefaultFencingScheme = false;
|
|
|
+ public static final int CREATE_DELETE_PERMS =
|
|
|
+ ZooDefs.Perms.CREATE | ZooDefs.Perms.DELETE;
|
|
|
+ private final String zkRootNodeAuthScheme =
|
|
|
+ new DigestAuthenticationProvider().getScheme();
|
|
|
+
|
|
|
+ private String zkRootNodeUsername;
|
|
|
+ private String zkRootNodePassword;
|
|
|
+
|
|
|
+ /**
|
|
|
+ * Given the {@link Configuration} and {@link ACL}s used (zkAcl) for
|
|
|
+ * ZooKeeper access, construct the {@link ACL}s for the store's root node.
|
|
|
+ * In the constructed {@link ACL}, all the users allowed by zkAcl are given
|
|
|
+ * rwa access, while the current RM has exclude create-delete access.
|
|
|
+ *
|
|
|
+ * To be called only when HA is enabled and the configuration doesn't set ACL
|
|
|
+ * for the root node.
|
|
|
+ */
|
|
|
+ @VisibleForTesting
|
|
|
+ @Private
|
|
|
+ @Unstable
|
|
|
+ protected List<ACL> constructZkRootNodeACL(
|
|
|
+ Configuration conf, List<ACL> sourceACLs) throws NoSuchAlgorithmException {
|
|
|
+ List<ACL> zkRootNodeAcl = new ArrayList<ACL>();
|
|
|
+ for (ACL acl : sourceACLs) {
|
|
|
+ zkRootNodeAcl.add(new ACL(
|
|
|
+ ZKUtil.removeSpecificPerms(acl.getPerms(), CREATE_DELETE_PERMS),
|
|
|
+ acl.getId()));
|
|
|
+ }
|
|
|
+
|
|
|
+ zkRootNodeUsername = HAUtil.getConfValueForRMInstance(
|
|
|
+ YarnConfiguration.RM_ADDRESS,
|
|
|
+ YarnConfiguration.DEFAULT_RM_ADDRESS, conf);
|
|
|
+ zkRootNodePassword = Long.toString(ResourceManager.getClusterTimeStamp());
|
|
|
+ Id rmId = new Id(zkRootNodeAuthScheme,
|
|
|
+ DigestAuthenticationProvider.generateDigest(
|
|
|
+ zkRootNodeUsername + ":" + zkRootNodePassword));
|
|
|
+ zkRootNodeAcl.add(new ACL(CREATE_DELETE_PERMS, rmId));
|
|
|
+ return zkRootNodeAcl;
|
|
|
+ }
|
|
|
+
|
|
|
@Override
|
|
|
public synchronized void initInternal(Configuration conf) throws Exception {
|
|
|
zkHostPort = conf.get(YarnConfiguration.ZK_RM_STATE_STORE_ADDRESS);
|
|
@@ -116,6 +174,29 @@ public class ZKRMStateStore extends RMStateStore {
|
|
|
zkRootNodePath = znodeWorkingPath + "/" + ROOT_ZNODE_NAME;
|
|
|
rmDTSecretManagerRoot = zkRootNodePath + "/" + RM_DT_SECRET_MANAGER_ROOT;
|
|
|
rmAppRoot = zkRootNodePath + "/" + RM_APP_ROOT;
|
|
|
+
|
|
|
+ /* Initialize fencing related paths, acls, and ops */
|
|
|
+ fencingNodePath = zkRootNodePath + "/" + FENCING_LOCK;
|
|
|
+ createFencingNodePathOp = Op.create(fencingNodePath, new byte[0], zkAcl,
|
|
|
+ CreateMode.PERSISTENT);
|
|
|
+ deleteFencingNodePathOp = Op.delete(fencingNodePath, -1);
|
|
|
+ if (HAUtil.isHAEnabled(conf)) {
|
|
|
+ String zkRootNodeAclConf = HAUtil.getConfValueForRMInstance
|
|
|
+ (YarnConfiguration.ZK_RM_STATE_STORE_ROOT_NODE_ACL, conf);
|
|
|
+ if (zkRootNodeAclConf != null) {
|
|
|
+ zkRootNodeAclConf = ZKUtil.resolveConfIndirection(zkRootNodeAclConf);
|
|
|
+ try {
|
|
|
+ zkRootNodeAcl = ZKUtil.parseACLs(zkRootNodeAclConf);
|
|
|
+ } catch (ZKUtil.BadAclFormatException bafe) {
|
|
|
+ LOG.error("Invalid format for " +
|
|
|
+ YarnConfiguration.ZK_RM_STATE_STORE_ROOT_NODE_ACL);
|
|
|
+ throw bafe;
|
|
|
+ }
|
|
|
+ } else {
|
|
|
+ useDefaultFencingScheme = true;
|
|
|
+ zkRootNodeAcl = constructZkRootNodeACL(conf, zkAcl);
|
|
|
+ }
|
|
|
+ }
|
|
|
}
|
|
|
|
|
|
@Override
|
|
@@ -126,20 +207,76 @@ public class ZKRMStateStore extends RMStateStore {
|
|
|
// ensure root dirs exist
|
|
|
createRootDir(znodeWorkingPath);
|
|
|
createRootDir(zkRootNodePath);
|
|
|
+ if (HAUtil.isHAEnabled(getConfig())){
|
|
|
+ fence();
|
|
|
+ }
|
|
|
createRootDir(rmDTSecretManagerRoot);
|
|
|
createRootDir(rmAppRoot);
|
|
|
}
|
|
|
|
|
|
- private void createRootDir(String rootPath) throws Exception {
|
|
|
+ private void createRootDir(final String rootPath) throws Exception {
|
|
|
+ // For root dirs, we shouldn't use the doMulti helper methods
|
|
|
try {
|
|
|
- createWithRetries(rootPath, null, zkAcl, CreateMode.PERSISTENT);
|
|
|
+ new ZKAction<String>() {
|
|
|
+ @Override
|
|
|
+ public String run() throws KeeperException, InterruptedException {
|
|
|
+ return zkClient.create(rootPath, null, zkAcl, CreateMode.PERSISTENT);
|
|
|
+ }
|
|
|
+ }.runWithRetries();
|
|
|
} catch (KeeperException ke) {
|
|
|
- if (ke.code() != Code.NODEEXISTS) {
|
|
|
+ if (ke.code() == Code.NODEEXISTS) {
|
|
|
+ LOG.debug(rootPath + "znode already exists!");
|
|
|
+ } else {
|
|
|
throw ke;
|
|
|
}
|
|
|
}
|
|
|
}
|
|
|
|
|
|
+ private void logRootNodeAcls(String prefix) throws KeeperException,
|
|
|
+ InterruptedException {
|
|
|
+ Stat getStat = new Stat();
|
|
|
+ List<ACL> getAcls = zkClient.getACL(zkRootNodePath, getStat);
|
|
|
+
|
|
|
+ StringBuilder builder = new StringBuilder();
|
|
|
+ builder.append(prefix);
|
|
|
+ for (ACL acl : getAcls) {
|
|
|
+ builder.append(acl.toString());
|
|
|
+ }
|
|
|
+ builder.append(getStat.toString());
|
|
|
+ LOG.debug(builder.toString());
|
|
|
+ }
|
|
|
+
|
|
|
+ private synchronized void fence() throws Exception {
|
|
|
+ if (LOG.isTraceEnabled()) {
|
|
|
+ logRootNodeAcls("Before fencing\n");
|
|
|
+ }
|
|
|
+
|
|
|
+ new ZKAction<Void>() {
|
|
|
+ @Override
|
|
|
+ public Void run() throws KeeperException, InterruptedException {
|
|
|
+ zkClient.setACL(zkRootNodePath, zkRootNodeAcl, -1);
|
|
|
+ return null;
|
|
|
+ }
|
|
|
+ }.runWithRetries();
|
|
|
+
|
|
|
+ // delete fencingnodepath
|
|
|
+ new ZKAction<Void>() {
|
|
|
+ @Override
|
|
|
+ public Void run() throws KeeperException, InterruptedException {
|
|
|
+ try {
|
|
|
+ zkClient.multi(Collections.singletonList(deleteFencingNodePathOp));
|
|
|
+ } catch (KeeperException.NoNodeException nne) {
|
|
|
+ LOG.info("Fencing node " + fencingNodePath + " doesn't exist to delete");
|
|
|
+ }
|
|
|
+ return null;
|
|
|
+ }
|
|
|
+ }.runWithRetries();
|
|
|
+
|
|
|
+ if (LOG.isTraceEnabled()) {
|
|
|
+ logRootNodeAcls("After fencing\n");
|
|
|
+ }
|
|
|
+ }
|
|
|
+
|
|
|
private synchronized void closeZkClients() throws IOException {
|
|
|
if (zkClient != null) {
|
|
|
try {
|
|
@@ -176,7 +313,8 @@ public class ZKRMStateStore extends RMStateStore {
|
|
|
|
|
|
private synchronized void loadRMDTSecretManagerState(RMState rmState)
|
|
|
throws Exception {
|
|
|
- List<String> childNodes = zkClient.getChildren(rmDTSecretManagerRoot, true);
|
|
|
+ List<String> childNodes =
|
|
|
+ getChildrenWithRetries(rmDTSecretManagerRoot, true);
|
|
|
|
|
|
for (String childNodeName : childNodes) {
|
|
|
if (childNodeName.startsWith(DELEGATION_TOKEN_SEQUENCE_NUMBER_PREFIX)) {
|
|
@@ -209,7 +347,7 @@ public class ZKRMStateStore extends RMStateStore {
|
|
|
}
|
|
|
|
|
|
private synchronized void loadRMAppState(RMState rmState) throws Exception {
|
|
|
- List<String> childNodes = zkClient.getChildren(rmAppRoot, true);
|
|
|
+ List<String> childNodes = getChildrenWithRetries(rmAppRoot, true);
|
|
|
List<ApplicationAttemptState> attempts =
|
|
|
new ArrayList<ApplicationAttemptState>();
|
|
|
for (String childNodeName : childNodes) {
|
|
@@ -466,6 +604,8 @@ public class ZKRMStateStore extends RMStateStore {
|
|
|
}
|
|
|
|
|
|
@VisibleForTesting
|
|
|
+ @Private
|
|
|
+ @Unstable
|
|
|
public synchronized void processWatchEvent(WatchedEvent event)
|
|
|
throws Exception {
|
|
|
Event.EventType eventType = event.getType();
|
|
@@ -506,65 +646,71 @@ public class ZKRMStateStore extends RMStateStore {
|
|
|
}
|
|
|
|
|
|
@VisibleForTesting
|
|
|
+ @Private
|
|
|
+ @Unstable
|
|
|
String getNodePath(String root, String nodeName) {
|
|
|
return (root + "/" + nodeName);
|
|
|
}
|
|
|
|
|
|
- @VisibleForTesting
|
|
|
- public String createWithRetries(
|
|
|
- final String path, final byte[] data, final List<ACL> acl,
|
|
|
- final CreateMode mode) throws Exception {
|
|
|
- return new ZKAction<String>() {
|
|
|
- @Override
|
|
|
- public String run() throws KeeperException, InterruptedException {
|
|
|
- return zkClient.create(path, data, acl, mode);
|
|
|
- }
|
|
|
- }.runWithRetries();
|
|
|
- }
|
|
|
-
|
|
|
- private void deleteWithRetries(final String path, final int version)
|
|
|
- throws Exception {
|
|
|
+ /**
|
|
|
+ * Helper method that creates fencing node, executes the passed operations,
|
|
|
+ * and deletes the fencing node.
|
|
|
+ */
|
|
|
+ private synchronized void doMultiWithRetries(
|
|
|
+ final List<Op> opList) throws Exception {
|
|
|
+ final List<Op> execOpList = new ArrayList<Op>(opList.size() + 2);
|
|
|
+ execOpList.add(createFencingNodePathOp);
|
|
|
+ execOpList.addAll(opList);
|
|
|
+ execOpList.add(deleteFencingNodePathOp);
|
|
|
new ZKAction<Void>() {
|
|
|
@Override
|
|
|
public Void run() throws KeeperException, InterruptedException {
|
|
|
- /**
|
|
|
- * Call exists() to leave a watch on the node denoted by path.
|
|
|
- * Delete node if exists. To pass the existence information to the
|
|
|
- * caller, call delete irrespective of whether node exists or not.
|
|
|
- */
|
|
|
- if (zkClient.exists(path, true) == null) {
|
|
|
- LOG.error("Trying to delete a path (" + path
|
|
|
- + ") that doesn't exist.");
|
|
|
- }
|
|
|
- zkClient.delete(path, version);
|
|
|
+ zkClient.multi(execOpList);
|
|
|
return null;
|
|
|
}
|
|
|
}.runWithRetries();
|
|
|
}
|
|
|
|
|
|
- private void doMultiWithRetries(final ArrayList<Op> opList) throws Exception {
|
|
|
- new ZKAction<Void>() {
|
|
|
- @Override
|
|
|
- public Void run() throws KeeperException, InterruptedException {
|
|
|
- zkClient.multi(opList);
|
|
|
- return null;
|
|
|
+ /**
|
|
|
+ * Helper method that creates fencing node, executes the passed operation,
|
|
|
+ * and deletes the fencing node.
|
|
|
+ */
|
|
|
+ private void doMultiWithRetries(final Op op) throws Exception {
|
|
|
+ doMultiWithRetries(Collections.singletonList(op));
|
|
|
+ }
|
|
|
+
|
|
|
+ @VisibleForTesting
|
|
|
+ @Private
|
|
|
+ @Unstable
|
|
|
+ public void createWithRetries(
|
|
|
+ final String path, final byte[] data, final List<ACL> acl,
|
|
|
+ final CreateMode mode) throws Exception {
|
|
|
+ doMultiWithRetries(Op.create(path, data, acl, mode));
|
|
|
+ }
|
|
|
+
|
|
|
+ private void deleteWithRetries(final String path, final int version)
|
|
|
+ throws Exception {
|
|
|
+ try {
|
|
|
+ doMultiWithRetries(Op.delete(path, version));
|
|
|
+ } catch (KeeperException.NoNodeException nne) {
|
|
|
+ // We tried to delete a node that doesn't exist
|
|
|
+ if (LOG.isDebugEnabled()) {
|
|
|
+ LOG.debug("Attempted to delete a non-existing znode " + path);
|
|
|
}
|
|
|
- }.runWithRetries();
|
|
|
+ }
|
|
|
}
|
|
|
|
|
|
@VisibleForTesting
|
|
|
+ @Private
|
|
|
+ @Unstable
|
|
|
public void setDataWithRetries(final String path, final byte[] data,
|
|
|
final int version) throws Exception {
|
|
|
- new ZKAction<Void>() {
|
|
|
- @Override
|
|
|
- public Void run() throws KeeperException, InterruptedException {
|
|
|
- zkClient.setData(path, data, version);
|
|
|
- return null;
|
|
|
- }
|
|
|
- }.runWithRetries();
|
|
|
+ doMultiWithRetries(Op.setData(path, data, version));
|
|
|
}
|
|
|
|
|
|
@VisibleForTesting
|
|
|
+ @Private
|
|
|
+ @Unstable
|
|
|
public byte[] getDataWithRetries(final String path, final boolean watch)
|
|
|
throws Exception {
|
|
|
return new ZKAction<byte[]>() {
|
|
@@ -576,6 +722,16 @@ public class ZKRMStateStore extends RMStateStore {
|
|
|
}.runWithRetries();
|
|
|
}
|
|
|
|
|
|
+ private List<String> getChildrenWithRetries(
|
|
|
+ final String path, final boolean watch) throws Exception {
|
|
|
+ return new ZKAction<List<String>>() {
|
|
|
+ @Override
|
|
|
+ List<String> run() throws KeeperException, InterruptedException {
|
|
|
+ return zkClient.getChildren(path, watch);
|
|
|
+ }
|
|
|
+ }.runWithRetries();
|
|
|
+ }
|
|
|
+
|
|
|
private abstract class ZKAction<T> {
|
|
|
// run() expects synchronization on ZKRMStateStore.this
|
|
|
abstract T run() throws KeeperException, InterruptedException;
|
|
@@ -596,11 +752,29 @@ public class ZKRMStateStore extends RMStateStore {
|
|
|
}
|
|
|
}
|
|
|
|
|
|
+ private boolean shouldRetry(Code code) {
|
|
|
+ switch (code) {
|
|
|
+ case CONNECTIONLOSS:
|
|
|
+ case OPERATIONTIMEOUT:
|
|
|
+ return true;
|
|
|
+ default:
|
|
|
+ break;
|
|
|
+ }
|
|
|
+ return false;
|
|
|
+ }
|
|
|
+
|
|
|
T runWithRetries() throws Exception {
|
|
|
int retry = 0;
|
|
|
while (true) {
|
|
|
try {
|
|
|
return runWithCheck();
|
|
|
+ } catch (KeeperException.NoAuthException nae) {
|
|
|
+ if (HAUtil.isHAEnabled(getConfig())) {
|
|
|
+ // NoAuthException possibly means that this store is fenced due to
|
|
|
+ // another RM becoming active. Even if not,
|
|
|
+ // it is safer to assume we have been fenced
|
|
|
+ throw new StoreFencedException();
|
|
|
+ }
|
|
|
} catch (KeeperException ke) {
|
|
|
if (shouldRetry(ke.code()) && ++retry < numRetries) {
|
|
|
continue;
|
|
@@ -611,17 +785,6 @@ public class ZKRMStateStore extends RMStateStore {
|
|
|
}
|
|
|
}
|
|
|
|
|
|
- private static boolean shouldRetry(Code code) {
|
|
|
- switch (code) {
|
|
|
- case CONNECTIONLOSS:
|
|
|
- case OPERATIONTIMEOUT:
|
|
|
- return true;
|
|
|
- default:
|
|
|
- break;
|
|
|
- }
|
|
|
- return false;
|
|
|
- }
|
|
|
-
|
|
|
private synchronized void createConnection()
|
|
|
throws IOException, InterruptedException {
|
|
|
closeZkClients();
|
|
@@ -629,6 +792,10 @@ public class ZKRMStateStore extends RMStateStore {
|
|
|
retries++) {
|
|
|
try {
|
|
|
zkClient = getNewZooKeeper();
|
|
|
+ if (useDefaultFencingScheme) {
|
|
|
+ zkClient.addAuthInfo(zkRootNodeAuthScheme,
|
|
|
+ (zkRootNodeUsername + ":" + zkRootNodePassword).getBytes());
|
|
|
+ }
|
|
|
} catch (IOException ioe) {
|
|
|
// Retry in case of network failures
|
|
|
LOG.info("Failed to connect to the ZooKeeper on attempt - " +
|
|
@@ -646,6 +813,8 @@ public class ZKRMStateStore extends RMStateStore {
|
|
|
|
|
|
// protected to mock for testing
|
|
|
@VisibleForTesting
|
|
|
+ @Private
|
|
|
+ @Unstable
|
|
|
protected synchronized ZooKeeper getNewZooKeeper()
|
|
|
throws IOException, InterruptedException {
|
|
|
ZooKeeper zk = new ZooKeeper(zkHostPort, zkSessionTimeout, null);
|