[hotfix] Replace HighAvailabilityOptions#HA_ZOOKEEPER_NAMESPACE with HA_CLUSTER_ID

上级 c4acbb83
......@@ -1031,7 +1031,7 @@ public final class ConfigConstants {
@Deprecated
public static final String HA_ZOOKEEPER_DIR_KEY = "high-availability.zookeeper.path.root";
/** @deprecated in favor of {@link HighAvailabilityOptions#HA_ZOOKEEPER_NAMESPACE}. */
/** @deprecated in favor of {@link HighAvailabilityOptions#HA_CLUSTER_ID}. */
@PublicEvolving
@Deprecated
public static final String HA_ZOOKEEPER_NAMESPACE_KEY = "high-availability.zookeeper.path.namespace";
......@@ -1788,7 +1788,7 @@ public final class ConfigConstants {
@Deprecated
public static final String DEFAULT_ZOOKEEPER_DIR_KEY = "/flink";
/** @deprecated in favor of {@link HighAvailabilityOptions#HA_ZOOKEEPER_NAMESPACE}. */
/** @deprecated in favor of {@link HighAvailabilityOptions#HA_CLUSTER_ID}. */
@Deprecated
public static final String DEFAULT_ZOOKEEPER_NAMESPACE_KEY = "/default";
......
......@@ -100,11 +100,6 @@ public class HighAvailabilityOptions {
.defaultValue("/flink")
.withDeprecatedKeys("recovery.zookeeper.path.root");
public static final ConfigOption<String> HA_ZOOKEEPER_NAMESPACE =
key("high-availability.zookeeper.path.namespace")
.noDefaultValue()
.withDeprecatedKeys("recovery.zookeeper.path.namespace");
public static final ConfigOption<String> HA_ZOOKEEPER_LATCH_PATH =
key("high-availability.zookeeper.path.latch")
.defaultValue("/leaderlatch")
......
......@@ -73,7 +73,7 @@ import java.util.Properties;
import java.util.concurrent.Callable;
import static org.apache.flink.client.cli.CliFrontendParser.ADDRESS_OPTION;
import static org.apache.flink.configuration.HighAvailabilityOptions.HA_ZOOKEEPER_NAMESPACE;
import static org.apache.flink.configuration.HighAvailabilityOptions.HA_CLUSTER_ID;
/**
* Class handling the command line interface to the YARN session.
......@@ -644,9 +644,9 @@ public class FlinkYarnSessionCli implements CustomCommandLine<YarnClusterClient>
String zkNamespace = cmd.hasOption(zookeeperNamespace.getOpt()) ?
cmd.getOptionValue(zookeeperNamespace.getOpt())
: yarnDescriptor.getFlinkConfiguration()
.getString(HA_ZOOKEEPER_NAMESPACE, cmd.getOptionValue(applicationId.getOpt()));
.getString(HA_CLUSTER_ID, cmd.getOptionValue(applicationId.getOpt()));
LOG.info("Going to use the ZK namespace: {}", zkNamespace);
yarnDescriptor.getFlinkConfiguration().setString(HA_ZOOKEEPER_NAMESPACE, zkNamespace);
yarnDescriptor.getFlinkConfiguration().setString(HA_CLUSTER_ID, zkNamespace);
try {
yarnCluster = yarnDescriptor.retrieve(cmd.getOptionValue(applicationId.getOpt()));
......
Markdown is supported
0% .
You are about to add 0 people to the discussion. Proceed with caution.
先完成此消息的编辑!
想要评论请 注册