YARN-10789. RM HA startup can fail due to race conditions in ZKConfigurationStore. Contributed by Tarun Parimi

This commit is contained in:
Szilard Nemeth 2021-07-29 19:21:58 +02:00
parent 797c595ba5
commit a272adc5fa

View File

@ -19,6 +19,7 @@
package org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.conf;
import org.apache.hadoop.thirdparty.com.google.common.annotations.VisibleForTesting;
import org.apache.zookeeper.KeeperException.NodeExistsException;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
import org.apache.hadoop.conf.Configuration;
@ -61,7 +62,8 @@ public class ZKConfigurationStore extends YarnConfigurationStore {
private static final String CONF_STORE_PATH = "CONF_STORE";
private static final String FENCING_PATH = "FENCING";
private static final String CONF_VERSION_PATH = "CONF_VERSION";
private static final String NODEEXISTS_MSG = "Encountered NodeExists error."
+ " Skipping znode creation since another RM has already created it";
private String zkVersionPath;
private String logsPath;
private String confStorePath;
@ -92,7 +94,11 @@ public void initialize(Configuration config, Configuration schedConf,
this.fencingNodePath = getNodePath(znodeParentPath, FENCING_PATH);
this.confVersionPath = getNodePath(znodeParentPath, CONF_VERSION_PATH);
zkManager.createRootDirRecursively(znodeParentPath, zkAcl);
try {
zkManager.createRootDirRecursively(znodeParentPath, zkAcl);
} catch(NodeExistsException e) {
LOG.warn(NODEEXISTS_MSG, e);
}
zkManager.delete(fencingNodePath);
if (createNewZkPath(logsPath)) {
@ -244,7 +250,12 @@ public List<LogMutation> getConfirmedConfHistory(long fromId) {
*/
private boolean createNewZkPath(String path) throws Exception {
if (!zkManager.exists(path)) {
zkManager.create(path);
try {
zkManager.create(path);
} catch(NodeExistsException e) {
LOG.warn(NODEEXISTS_MSG, e);
return false;
}
return true;
} else {
return false;
@ -279,8 +290,12 @@ private void safeSetZkData(String path, Object data) throws Exception {
@VisibleForTesting
protected void safeCreateZkData(String path, byte[] data) throws Exception {
zkManager.safeCreate(path, data, zkAcl, CreateMode.PERSISTENT,
zkAcl, fencingNodePath);
try {
zkManager.safeCreate(path, data, zkAcl, CreateMode.PERSISTENT,
zkAcl, fencingNodePath);
} catch(NodeExistsException e) {
LOG.warn(NODEEXISTS_MSG, e);
}
}
private static String getNodePath(String root, String nodeName) {