Github user EronWright commented on a diff in the pull request:

    https://github.com/apache/flink/pull/4261#discussion_r128109508
  
    --- Diff: 
flink-runtime/src/main/java/org/apache/flink/runtime/entrypoint/ClusterEntrypoint.java
 ---
    @@ -0,0 +1,247 @@
    +/*
    + * Licensed to the Apache Software Foundation (ASF) under one
    + * or more contributor license agreements.  See the NOTICE file
    + * distributed with this work for additional information
    + * regarding copyright ownership.  The ASF licenses this file
    + * to you under the Apache License, Version 2.0 (the
    + * "License"); you may not use this file except in compliance
    + * with the License.  You may obtain a copy of the License at
    + *
    + *     http://www.apache.org/licenses/LICENSE-2.0
    + *
    + * Unless required by applicable law or agreed to in writing, software
    + * distributed under the License is distributed on an "AS IS" BASIS,
    + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
    + * See the License for the specific language governing permissions and
    + * limitations under the License.
    + */
    +
    +package org.apache.flink.runtime.entrypoint;
    +
    +import org.apache.flink.api.common.time.Time;
    +import org.apache.flink.api.java.utils.ParameterTool;
    +import org.apache.flink.configuration.Configuration;
    +import org.apache.flink.configuration.GlobalConfiguration;
    +import org.apache.flink.configuration.JobManagerOptions;
    +import org.apache.flink.runtime.akka.AkkaUtils;
    +import org.apache.flink.runtime.blob.BlobServer;
    +import org.apache.flink.runtime.clusterframework.BootstrapTools;
    +import org.apache.flink.runtime.heartbeat.HeartbeatServices;
    +import org.apache.flink.runtime.highavailability.HighAvailabilityServices;
    +import 
org.apache.flink.runtime.highavailability.HighAvailabilityServicesUtils;
    +import org.apache.flink.runtime.metrics.MetricRegistry;
    +import org.apache.flink.runtime.metrics.MetricRegistryConfiguration;
    +import org.apache.flink.runtime.rpc.FatalErrorHandler;
    +import org.apache.flink.runtime.rpc.RpcService;
    +import org.apache.flink.runtime.rpc.akka.AkkaRpcService;
    +import org.apache.flink.runtime.security.SecurityContext;
    +import org.apache.flink.runtime.security.SecurityUtils;
    +import org.apache.flink.util.ExceptionUtils;
    +import org.apache.flink.util.FlinkException;
    +
    +import akka.actor.ActorSystem;
    +import org.slf4j.Logger;
    +import org.slf4j.LoggerFactory;
    +
    +import javax.annotation.concurrent.GuardedBy;
    +
    +import java.util.concurrent.Callable;
    +import java.util.concurrent.Executor;
    +
    +import scala.concurrent.duration.FiniteDuration;
    +
    +/**
    + * Base class for the Flink cluster entry points.
    + *
    + * <p>Specialization of this class can be used for the session mode and 
the per-job mode
    + */
    +public abstract class ClusterEntrypoint implements FatalErrorHandler {
    +
    +   protected static final Logger LOG = 
LoggerFactory.getLogger(ClusterEntrypoint.class);
    +
    +   protected static final int SUCCESS_RETURN_CODE = 0;
    +   protected static final int STARTUP_FAILURE_RETURN_CODE = 1;
    +   protected static final int RUNTIME_FAILURE_RETURN_CODE = 2;
    +
    +   /** The lock to guard startup / shutdown / manipulation methods. */
    +   private final Object lock = new Object();
    +
    +   @GuardedBy("lock")
    +   private MetricRegistry metricRegistry = null;
    +
    +   @GuardedBy("lock")
    +   private HighAvailabilityServices haServices = null;
    +
    +   @GuardedBy("lock")
    +   private BlobServer blobServer = null;
    +
    +   @GuardedBy("lock")
    +   private HeartbeatServices heartbeatServices = null;
    +
    +   @GuardedBy("lock")
    +   private RpcService commonRpcService = null;
    +
    +   protected void startCluster(String[] args) {
    +           final ClusterConfiguration clusterConfiguration = 
parseArguments(args);
    +
    +           final Configuration configuration = 
loadConfiguration(clusterConfiguration);
    +
    +           try {
    +                   SecurityContext securityContext = 
installSecurityContext(configuration);
    +
    +                   securityContext.runSecured(new Callable<Void>() {
    +                           @Override
    +                           public Void call() throws Exception {
    +                                   runCluster(configuration);
    +
    +                                   return null;
    +                           }
    +                   });
    +           } catch (Throwable t) {
    +                   LOG.error("Cluster initialization failed.", t);
    +
    +                   try {
    +                           shutDown(false);
    +                   } catch (Throwable st) {
    +                           LOG.error("Could not properly shut down cluster 
entrypoint.", st);
    +                   }
    +
    +                   System.exit(STARTUP_FAILURE_RETURN_CODE);
    +           }
    +   }
    +
    +   protected ClusterConfiguration parseArguments(String[] args) {
    +           ParameterTool parameterTool = ParameterTool.fromArgs(args);
    +
    +           final String configDir = parameterTool.get("configDir", "");
    +
    +           return new ClusterConfiguration(configDir);
    +   }
    +
    +   protected Configuration loadConfiguration(ClusterConfiguration 
clusterConfiguration) {
    --- End diff --
    
    Consider supporting dynamic properties, e.g. `jobmanager.sh 
-Djobmanager.rpc.port=1234` to assist with YARN/Mesos scenarios, where 
adjusting the `flink-conf.yaml` might not be feasible.
    
    As seen in `MesosTaskManagerRunner`:
    ```
    final Configuration dynamicProperties = 
BootstrapTools.parseDynamicProperties(cmd);
    GlobalConfiguration.setDynamicProperties(dynamicProperties);
    ```
    



---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---

Reply via email to