[hotfix] Enable standalone HA mode by choosing HA port range
authorTill Rohrmann <trohrmann@apache.org>
Fri, 2 Mar 2018 19:31:36 +0000 (20:31 +0100)
committerTill Rohrmann <trohrmann@apache.org>
Fri, 2 Mar 2018 22:43:31 +0000 (23:43 +0100)
flink-runtime/src/main/java/org/apache/flink/runtime/entrypoint/ClusterEntrypoint.java

index 19781f8..07b3b68 100755 (executable)
@@ -24,6 +24,7 @@ import org.apache.flink.configuration.ConfigOption;
 import org.apache.flink.configuration.ConfigOptions;
 import org.apache.flink.configuration.Configuration;
 import org.apache.flink.configuration.GlobalConfiguration;
+import org.apache.flink.configuration.HighAvailabilityOptions;
 import org.apache.flink.configuration.JobManagerOptions;
 import org.apache.flink.configuration.RestOptions;
 import org.apache.flink.configuration.WebOptions;
@@ -59,6 +60,7 @@ import org.apache.flink.runtime.rpc.akka.AkkaRpcService;
 import org.apache.flink.runtime.security.SecurityConfiguration;
 import org.apache.flink.runtime.security.SecurityContext;
 import org.apache.flink.runtime.security.SecurityUtils;
+import org.apache.flink.runtime.util.ZooKeeperUtils;
 import org.apache.flink.runtime.webmonitor.WebMonitorEndpoint;
 import org.apache.flink.runtime.webmonitor.retriever.LeaderGatewayRetriever;
 import org.apache.flink.runtime.webmonitor.retriever.MetricQueryServiceRetriever;
@@ -354,7 +356,11 @@ public abstract class ClusterEntrypoint implements FatalErrorHandler {
         * @return Port range for the common {@link RpcService}
         */
        protected String getRPCPortRange(Configuration configuration) {
-               return String.valueOf(configuration.getInteger(JobManagerOptions.PORT));
+               if (ZooKeeperUtils.isZooKeeperRecoveryMode(configuration)) {
+                       return configuration.getString(HighAvailabilityOptions.HA_JOB_MANAGER_PORT_RANGE);
+               } else {
+                       return String.valueOf(configuration.getInteger(JobManagerOptions.PORT));
+               }
        }
 
        protected RpcService createRpcService(