Return-Path: X-Original-To: archive-asf-public-internal@cust-asf2.ponee.io Delivered-To: archive-asf-public-internal@cust-asf2.ponee.io Received: from cust-asf.ponee.io (cust-asf.ponee.io [163.172.22.183]) by cust-asf2.ponee.io (Postfix) with ESMTP id 57DD8200B65 for ; Wed, 17 Aug 2016 14:11:09 +0200 (CEST) Received: by cust-asf.ponee.io (Postfix) id 5683A160A8C; Wed, 17 Aug 2016 12:11:09 +0000 (UTC) Delivered-To: archive-asf-public@cust-asf.ponee.io Received: from mail.apache.org (hermes.apache.org [140.211.11.3]) by cust-asf.ponee.io (Postfix) with SMTP id 4BAA3160A86 for ; Wed, 17 Aug 2016 14:11:08 +0200 (CEST) Received: (qmail 66864 invoked by uid 500); 17 Aug 2016 12:11:07 -0000 Mailing-List: contact issues-help@flink.apache.org; run by ezmlm Precedence: bulk List-Help: List-Unsubscribe: List-Post: List-Id: Reply-To: dev@flink.apache.org Delivered-To: mailing list issues@flink.apache.org Received: (qmail 66855 invoked by uid 99); 17 Aug 2016 12:11:07 -0000 Received: from pnap-us-west-generic-nat.apache.org (HELO spamd2-us-west.apache.org) (209.188.14.142) by apache.org (qpsmtpd/0.29) with ESMTP; Wed, 17 Aug 2016 12:11:07 +0000 Received: from localhost (localhost [127.0.0.1]) by spamd2-us-west.apache.org (ASF Mail Server at spamd2-us-west.apache.org) with ESMTP id 1A77A1A0272 for ; Wed, 17 Aug 2016 12:11:07 +0000 (UTC) X-Virus-Scanned: Debian amavisd-new at spamd2-us-west.apache.org X-Spam-Flag: NO X-Spam-Score: -4.646 X-Spam-Level: X-Spam-Status: No, score=-4.646 tagged_above=-999 required=6.31 tests=[KAM_ASCII_DIVIDERS=0.8, KAM_LAZY_DOMAIN_SECURITY=1, RCVD_IN_DNSWL_HI=-5, RCVD_IN_MSPIKE_H3=-0.01, RCVD_IN_MSPIKE_WL=-0.01, RP_MATCHES_RCVD=-1.426] autolearn=disabled Received: from mx1-lw-eu.apache.org ([10.40.0.8]) by localhost (spamd2-us-west.apache.org [10.40.0.9]) (amavisd-new, port 10024) with ESMTP id aRsBIRn1yAgo for ; Wed, 17 Aug 2016 12:11:05 +0000 (UTC) Received: from mail.apache.org (hermes.apache.org [140.211.11.3]) by mx1-lw-eu.apache.org (ASF Mail Server at mx1-lw-eu.apache.org) with SMTP id ED6C15FBD0 for ; Wed, 17 Aug 2016 12:11:03 +0000 (UTC) Received: (qmail 66818 invoked by uid 99); 17 Aug 2016 12:11:03 -0000 Received: from git1-us-west.apache.org (HELO git1-us-west.apache.org) (140.211.11.23) by apache.org (qpsmtpd/0.29) with ESMTP; Wed, 17 Aug 2016 12:11:03 +0000 Received: by git1-us-west.apache.org (ASF Mail Server at git1-us-west.apache.org, from userid 33) id 18AFCE0A51; Wed, 17 Aug 2016 12:11:03 +0000 (UTC) From: StephanEwen To: issues@flink.incubator.apache.org Reply-To: issues@flink.incubator.apache.org References: In-Reply-To: Subject: [GitHub] flink pull request #2315: [FLINK-1984] Integrate Flink with Apache Mesos (T1... Content-Type: text/plain Message-Id: <20160817121103.18AFCE0A51@git1-us-west.apache.org> Date: Wed, 17 Aug 2016 12:11:03 +0000 (UTC) archived-at: Wed, 17 Aug 2016 12:11:09 -0000 Github user StephanEwen commented on a diff in the pull request: https://github.com/apache/flink/pull/2315#discussion_r75109862 --- Diff: flink-mesos/src/main/java/org/apache/flink/mesos/runtime/clusterframework/MesosApplicationMasterRunner.java --- @@ -0,0 +1,618 @@ +/* + * 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.mesos.runtime.clusterframework; + +import akka.actor.ActorRef; +import akka.actor.ActorSystem; +import akka.actor.Props; + +import org.apache.curator.framework.CuratorFramework; +import org.apache.flink.configuration.ConfigConstants; +import org.apache.flink.configuration.Configuration; +import org.apache.flink.configuration.GlobalConfiguration; +import org.apache.flink.configuration.IllegalConfigurationException; +import org.apache.flink.mesos.cli.FlinkMesosSessionCli; +import org.apache.flink.mesos.runtime.clusterframework.store.MesosWorkerStore; +import org.apache.flink.mesos.runtime.clusterframework.store.StandaloneMesosWorkerStore; +import org.apache.flink.mesos.runtime.clusterframework.store.ZooKeeperMesosWorkerStore; +import org.apache.flink.mesos.util.MesosArtifactServer; +import org.apache.flink.mesos.util.MesosConfiguration; +import org.apache.flink.mesos.util.ZooKeeperUtils; +import org.apache.flink.runtime.akka.AkkaUtils; +import org.apache.flink.runtime.clusterframework.BootstrapTools; +import org.apache.flink.runtime.clusterframework.ContaineredTaskManagerParameters; +import org.apache.flink.runtime.jobmanager.JobManager; +import org.apache.flink.runtime.jobmanager.MemoryArchivist; +import org.apache.flink.runtime.jobmanager.RecoveryMode; +import org.apache.flink.runtime.leaderretrieval.LeaderRetrievalService; +import org.apache.flink.runtime.process.ProcessReaper; +import org.apache.flink.runtime.taskmanager.TaskManager; +import org.apache.flink.runtime.util.EnvironmentInformation; +import org.apache.flink.runtime.util.LeaderRetrievalUtils; +import org.apache.flink.runtime.util.SignalHandler; +import org.apache.flink.runtime.webmonitor.WebMonitor; + +import org.apache.hadoop.security.UserGroupInformation; + +import org.apache.mesos.Protos; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import scala.Option; +import scala.concurrent.duration.Duration; +import scala.concurrent.duration.FiniteDuration; + +import java.io.File; +import java.net.InetAddress; +import java.net.URL; +import java.security.PrivilegedAction; +import java.util.Map; +import java.util.UUID; +import java.util.concurrent.TimeUnit; + +import static org.apache.flink.mesos.Utils.uri; +import static org.apache.flink.mesos.Utils.variable; + +/** + * This class is the executable entry point for the Mesos Application Master. + * It starts actor system and the actors for {@link org.apache.flink.runtime.jobmanager.JobManager} + * and {@link MesosFlinkResourceManager}. + * + * The JobManager handles Flink job execution, while the MesosFlinkResourceManager handles container + * allocation and failure detection. + */ +public class MesosApplicationMasterRunner { + /** Logger */ + protected static final Logger LOG = LoggerFactory.getLogger(MesosApplicationMasterRunner.class); + + /** The maximum time that TaskManagers may be waiting to register at the JobManager, + * before they quit */ + private static final FiniteDuration TASKMANAGER_REGISTRATION_TIMEOUT = new FiniteDuration(5, TimeUnit.MINUTES); + + /** The process environment variables */ + private static final Map ENV = System.getenv(); + + /** The exit code returned if the initialization of the application master failed */ + private static final int INIT_ERROR_EXIT_CODE = 31; + + /** The exit code returned if the process exits because a critical actor died */ + private static final int ACTOR_DIED_EXIT_CODE = 32; + + // ------------------------------------------------------------------------ + // Program entry point + // ------------------------------------------------------------------------ + + /** + * The entry point for the Mesos AppMaster. + * + * @param args The command line arguments. + */ + public static void main(String[] args) { + EnvironmentInformation.logEnvironmentInfo(LOG, "Mesos AppMaster", args); + SignalHandler.register(LOG); + + // run and exit with the proper return code + int returnCode = new MesosApplicationMasterRunner().run(args); + System.exit(returnCode); + } + + /** + * The instance entry point for the Mesos AppMaster. Obtains user group + * information and calls the main work method {@link #runPrivileged()} as a + * privileged action. + * + * @param args The command line arguments. + * @return The process exit code. + */ + protected int run(String[] args) { + try { + LOG.debug("All environment variables: {}", ENV); + + final UserGroupInformation currentUser; + try { + currentUser = UserGroupInformation.getCurrentUser(); + } catch (Throwable t) { + throw new Exception("Cannot access UserGroupInformation information for current user", t); + } + + LOG.info("Running Flink as user {}", currentUser.getShortUserName()); + + // run the actual work in a secured privileged action + return currentUser.doAs(new PrivilegedAction() { + @Override + public Integer run() { + return runPrivileged(); + } + }); + } + catch (Throwable t) { + // make sure that everything whatever ends up in the log + LOG.error("Mesos AppMaster initialization failed", t); + return INIT_ERROR_EXIT_CODE; + } + } + + // ------------------------------------------------------------------------ + // Core work method + // ------------------------------------------------------------------------ + + /** + * The main work method, must run as a privileged action. + * + * @return The return code for the Java process. + */ + protected int runPrivileged() { + + ActorSystem actorSystem = null; + WebMonitor webMonitor = null; + MesosArtifactServer artifactServer = null; + + try { + // ------- (1) load and parse / validate all configurations ------- + + // loading all config values here has the advantage that the program fails fast, if any + // configuration problem occurs + + final String workingDir = ENV.get(MesosConfigKeys.ENV_MESOS_SANDBOX); + require(workingDir != null, "Sandbox directory variable (%s) not set", MesosConfigKeys.ENV_MESOS_SANDBOX); + + final String sessionID = ENV.get(MesosConfigKeys.ENV_SESSION_ID); + require(sessionID != null, "Session ID (%s) not set", MesosConfigKeys.ENV_SESSION_ID); + + // Note that we use the "appMasterHostname" given by the system, to make sure + // we use the hostnames consistently throughout akka. + // for akka "localhost" and "localhost.localdomain" are different actors. + final String appMasterHostname = InetAddress.getLocalHost().getHostName(); + + // Flink configuration + final Configuration dynamicProperties = + FlinkMesosSessionCli.decodeDynamicProperties(ENV.get(MesosConfigKeys.ENV_DYNAMIC_PROPERTIES)); + LOG.debug("Mesos dynamic properties: {}", dynamicProperties); + + final Configuration config = createConfiguration(workingDir, dynamicProperties); + + // Mesos configuration + final MesosConfiguration mesosConfig = createMesosConfig(config, appMasterHostname); + + // environment values related to TM + final int taskManagerContainerMemory; + final int numInitialTaskManagers; + final int slotsPerTaskManager; + + try { + taskManagerContainerMemory = Integer.parseInt(ENV.get(MesosConfigKeys.ENV_TM_MEMORY)); + } catch (NumberFormatException e) { + throw new RuntimeException("Invalid value for " + MesosConfigKeys.ENV_TM_MEMORY + " : " + + e.getMessage()); + } + try { + numInitialTaskManagers = Integer.parseInt(ENV.get(MesosConfigKeys.ENV_TM_COUNT)); + } catch (NumberFormatException e) { + throw new RuntimeException("Invalid value for " + MesosConfigKeys.ENV_TM_COUNT + " : " + + e.getMessage()); + } + try { + slotsPerTaskManager = Integer.parseInt(ENV.get(MesosConfigKeys.ENV_SLOTS)); + } catch (NumberFormatException e) { + throw new RuntimeException("Invalid value for " + MesosConfigKeys.ENV_SLOTS + " : " + + e.getMessage()); + } + + final ContaineredTaskManagerParameters containeredParameters = + ContaineredTaskManagerParameters.create(config, taskManagerContainerMemory, slotsPerTaskManager); + + final MesosTaskManagerParameters taskManagerParameters = + MesosTaskManagerParameters.create(config, containeredParameters); + + LOG.info("TaskManagers will be created with {} task slots", + taskManagerParameters.containeredParameters().numSlots()); + LOG.info("TaskManagers will be started with container size {} MB, JVM heap size {} MB, " + + "JVM direct memory limit {} MB, {} cpus", + taskManagerParameters.containeredParameters().taskManagerTotalMemoryMB(), + taskManagerParameters.containeredParameters().taskManagerHeapSizeMB(), + taskManagerParameters.containeredParameters().taskManagerDirectMemoryLimitMB(), + taskManagerParameters.cpus()); + + // JM endpoint, which should be explicitly configured by the dispatcher (based on acquired net resources) + final int listeningPort = config.getInteger(ConfigConstants.JOB_MANAGER_IPC_PORT_KEY, + ConfigConstants.DEFAULT_JOB_MANAGER_IPC_PORT); + require(listeningPort >= 0 && listeningPort <= 65536, "Config parameter \"" + + ConfigConstants.JOB_MANAGER_IPC_PORT_KEY + "\" is invalid, it must be between 0 and 65536"); + + // ----------------- (2) start the actor system ------------------- + + // try to start the actor system, JobManager and JobManager actor system + // using the configured address and ports + actorSystem = BootstrapTools.startActorSystem(config, appMasterHostname, listeningPort, LOG); + + final String akkaHostname = AkkaUtils.getAddress(actorSystem).host().get(); + final int akkaPort = (Integer) AkkaUtils.getAddress(actorSystem).port().get(); + + LOG.info("Actor system bound to hostname {}.", akkaHostname); + + // try to start the artifact server + LOG.debug("Starting Artifact Server"); + final int artifactServerPort = config.getInteger(ConfigConstants.MESOS_ARTIFACT_SERVER_PORT_KEY, + ConfigConstants.DEFAULT_MESOS_ARTIFACT_SERVER_PORT); + artifactServer = new MesosArtifactServer(sessionID, akkaHostname, artifactServerPort); + + // ----------------- (3) Generate the configuration for the TaskManagers ------------------- + + final Configuration taskManagerConfig = BootstrapTools.generateTaskManagerConfiguration( + config, akkaHostname, akkaPort, slotsPerTaskManager, TASKMANAGER_REGISTRATION_TIMEOUT); + LOG.debug("TaskManager configuration: {}", taskManagerConfig); + + final Protos.TaskInfo.Builder taskManagerContext = createTaskManagerContext( + config, mesosConfig, ENV, --- End diff -- We would actually need to define that style somewhere. I also do the variable version once in a while. Whatever helps readability is very subjective, so we need a standard (or deliberately not standardize on that). --- 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 infrastructure@apache.org or file a JIRA ticket with INFRA. ---