You are viewing a plain text version of this content. The canonical link for it is here.
Posted to issues@flink.apache.org by wangzhijiang999 <gi...@git.apache.org> on 2016/08/22 04:00:12 UTC

[GitHub] flink pull request #2400: [FLINK-4363] Implement TaskManager basic startup o...

GitHub user wangzhijiang999 opened a pull request:

    https://github.com/apache/flink/pull/2400

    [FLINK-4363] Implement TaskManager basic startup of all components in java

    Thanks for contributing to Apache Flink. Before you open your pull request, please take the following check list into consideration.
    If your changes take all of the items into account, feel free to open your pull request. For more information and/or questions please refer to the [How To Contribute guide](http://flink.apache.org/how-to-contribute.html).
    In addition to going through the list, please provide a meaningful description of your changes.
    
    - [ ] General
      - The pull request references the related JIRA issue ("[FLINK-XXX] Jira title text")
      - The pull request addresses only one issue
      - Each commit in the PR has a meaningful commit message (including the JIRA id)
    
    - [ ] Documentation
      - Documentation has been added for new functionality
      - Old documentation affected by the pull request has been updated
      - JavaDoc for public methods has been added
    
    - [ ] Tests & Build
      - Functionality added by the pull request is covered by tests
      - `mvn clean verify` has been executed successfully locally or a Travis build has passed


You can merge this pull request into a Git repository by running:

    $ git pull https://github.com/alibaba/flink jira-4363

Alternatively you can review and apply these changes as the patch at:

    https://github.com/apache/flink/pull/2400.patch

To close this pull request, make a commit to your master/trunk branch
with (at least) the following in the commit message:

    This closes #2400
    
----
commit afc504f37ec873b3be30320c3bcbc93fcbe1aec3
Author: \u6dd8\u6c5f <ta...@alibaba-inc.com>
Date:   2016-08-19T09:32:10Z

    implement taskmanager startup basic components in java [#FLINK-4363]

commit bba0f679d3b2f7b779c18b13fefd4b50edbdfb69
Author: \u6dd8\u6c5f <ta...@alibaba-inc.com>
Date:   2016-08-22T03:13:02Z

    Merge branch 'flip-6' of https://github.com/apache/flink into flink-tm_startup

commit 22da4e24e155f309d2705bbb5a6d1a64744dc4e3
Author: \u6dd8\u6c5f <ta...@alibaba-inc.com>
Date:   2016-08-22T03:52:05Z

    [FLINK-4363] Implement TaskManager basic startup of all components in java

----


---
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.
---

[GitHub] flink pull request #2400: [FLINK-4363] Implement TaskManager basic startup o...

Posted by mxm <gi...@git.apache.org>.
Github user mxm commented on a diff in the pull request:

    https://github.com/apache/flink/pull/2400#discussion_r76084721
  
    --- Diff: flink-runtime/src/main/java/org/apache/flink/runtime/rpc/taskexecutor/TaskExecutor.java ---
    @@ -36,27 +82,617 @@
      */
     public class TaskExecutor extends RpcEndpoint<TaskExecutorGateway> {
     
    -	/** The unique resource ID of this TaskExecutor */
    +	private static final Logger LOG = LoggerFactory.getLogger(TaskExecutor.class);
    +
    +	/** The unique resource ID of this TaskManager */
     	private final ResourceID resourceID;
     
     	/** The access to the leader election and metadata storage services */
     	private final HighAvailabilityServices haServices;
     
    -	// --------- resource manager --------
    +	/** The task manager configuration */
    +	private final TaskExecutorConfiguration taskExecutorConfig;
    +
    +	/** The connection information of the task manager */
    +	private final InstanceConnectionInfo connectionInfo;
    +
    +	/** The I/O manager component in the task manager */
    +	private final IOManager ioManager;
    +
    +	/** The memory manager component in the task manager */
    +	private final MemoryManager memoryManager;
    +
    +	/** The network component in the task manager */
    +	private final NetworkEnvironment networkEnvironment;
    +
    +	/** The number of slots in the task manager, should be 1 for YARN */
    +	private final int numberOfSlots;
     
    +	// --------- resource manager --------
     	private TaskExecutorToResourceManagerConnection resourceManagerConnection;
     
     	// ------------------------------------------------------------------------
     
     	public TaskExecutor(
    +			TaskExecutorConfiguration taskExecutorConfig,
    +			ResourceID resourceID,
    +			InstanceConnectionInfo connectionInfo,
    +			MemoryManager memoryManager,
    +			IOManager ioManager,
    +			NetworkEnvironment networkEnvironment,
    +			int numberOfSlots,
     			RpcService rpcService,
    -			HighAvailabilityServices haServices,
    -			ResourceID resourceID) {
    +			HighAvailabilityServices haServices) {
     
     		super(rpcService);
     
    -		this.haServices = checkNotNull(haServices);
    +		this.taskExecutorConfig = checkNotNull(taskExecutorConfig);
     		this.resourceID = checkNotNull(resourceID);
    +		this.connectionInfo = checkNotNull(connectionInfo);
    +		this.memoryManager = checkNotNull(memoryManager);
    +		this.ioManager = checkNotNull(ioManager);
    +		this.networkEnvironment = checkNotNull(networkEnvironment);
    +		this.numberOfSlots = checkNotNull(numberOfSlots);
    +		this.haServices = checkNotNull(haServices);
    +	}
    +
    +	/**
    +	 * Starts and runs the TaskManager.
    +	 * <p/>
    +	 * This method first tries to select the network interface to use for the TaskManager
    +	 * communication. The network interface is used both for the actor communication
    +	 * (coordination) as well as for the data exchange between task managers. Unless
    +	 * the hostname/interface is explicitly configured in the configuration, this
    +	 * method will try out various interfaces and methods to connect to the JobManager
    +	 * and select the one where the connection attempt is successful.
    +	 * <p/>
    +	 * After selecting the network interface, this method brings up an actor system
    +	 * for the TaskManager and its actors, starts the TaskManager's services
    +	 * (library cache, shuffle network stack, ...), and starts the TaskManager itself.
    +	 *
    +	 * @param configuration    The configuration for the TaskManager.
    +	 * @param resourceID       The id of the resource which the task manager will run on.
    +	 */
    +	public static void selectNetworkInterfaceAndRunTaskManager(
    +		Configuration configuration,
    +		ResourceID resourceID) throws Exception {
    +
    +		InetSocketAddress taskManagerAddress = selectNetworkInterfaceAndPort(configuration);
    +
    +		runTaskManager(taskManagerAddress.getHostName(), resourceID, taskManagerAddress.getPort(), configuration);
    +	}
    +
    +	private static InetSocketAddress selectNetworkInterfaceAndPort(Configuration configuration)
    +		throws Exception {
    +		String taskManagerHostname = configuration.getString(ConfigConstants.TASK_MANAGER_HOSTNAME_KEY, null);
    +		if (taskManagerHostname != null) {
    +			LOG.info("Using configured hostname/address for TaskManager: " + taskManagerHostname);
    +		} else {
    +			LeaderRetrievalService leaderRetrievalService = LeaderRetrievalUtils.createLeaderRetrievalService(configuration);
    +			FiniteDuration lookupTimeout = AkkaUtils.getLookupTimeout(configuration);
    +
    +			InetAddress taskManagerAddress = LeaderRetrievalUtils.findConnectingAddress(leaderRetrievalService, lookupTimeout);
    +			taskManagerHostname = taskManagerAddress.getHostName();
    +			LOG.info("TaskManager will use hostname/address '{}' ({}) for communication.",
    +				taskManagerHostname, taskManagerAddress.getHostAddress());
    +		}
    +
    +		// if no task manager port has been configured, use 0 (system will pick any free port)
    +		int actorSystemPort = configuration.getInteger(ConfigConstants.TASK_MANAGER_IPC_PORT_KEY, 0);
    +		if (actorSystemPort < 0 || actorSystemPort > 65535) {
    +			throw new IllegalConfigurationException("Invalid value for '" +
    +				ConfigConstants.TASK_MANAGER_IPC_PORT_KEY +
    +				"' (port for the TaskManager actor system) : " + actorSystemPort +
    +				" - Leave config parameter empty or use 0 to let the system choose a port automatically.");
    +		}
    +
    +		return new InetSocketAddress(taskManagerHostname, actorSystemPort);
    +	}
    +
    +	/**
    +	 * Starts and runs the TaskManager. Brings up an actor system for the TaskManager and its
    +	 * actors, starts the TaskManager's services (library cache, shuffle network stack, ...),
    +	 * and starts the TaskManager itself.
    +	 * <p/>
    +	 * This method will also spawn a process reaper for the TaskManager (kill the process if
    +	 * the actor fails) and optionally start the JVM memory logging thread.
    +	 *
    +	 * @param taskManagerHostname The hostname/address of the interface where the actor system
    +	 *                            will communicate.
    +	 * @param resourceID          The id of the resource which the task manager will run on.
    +	 * @param actorSystemPort   The port at which the actor system will communicate.
    +	 * @param configuration       The configuration for the TaskManager.
    +	 */
    +	private static void runTaskManager(
    +		String taskManagerHostname,
    +		ResourceID resourceID,
    +		int actorSystemPort,
    +		final Configuration configuration) throws Exception {
    +
    +		LOG.info("Starting TaskManager");
    +
    +		// Bring up the TaskManager actor system first, bind it to the given address.
    +
    +		LOG.info("Starting TaskManager actor system at " +
    +			NetUtils.hostAndPortToUrlString(taskManagerHostname, actorSystemPort));
    +
    +		final ActorSystem taskManagerSystem;
    +		try {
    +			Tuple2<String, Object> address = new Tuple2<String, Object>(taskManagerHostname, actorSystemPort);
    +			Config akkaConfig = AkkaUtils.getAkkaConfig(configuration, new Some<>(address));
    +			LOG.debug("Using akka configuration\n " + akkaConfig);
    +			taskManagerSystem = AkkaUtils.createActorSystem(akkaConfig);
    +		} catch (Throwable t) {
    +			if (t instanceof org.jboss.netty.channel.ChannelException) {
    +				Throwable cause = t.getCause();
    +				if (cause != null && t.getCause() instanceof java.net.BindException) {
    +					String address = NetUtils.hostAndPortToUrlString(taskManagerHostname, actorSystemPort);
    +					throw new IOException("Unable to bind TaskManager actor system to address " +
    +						address + " - " + cause.getMessage(), t);
    +				}
    +			}
    +			throw new Exception("Could not create TaskManager actor system", t);
    +		}
    +
    +		// start akka rpc service based on actor system
    +		Timeout timeout = new Timeout(AkkaUtils.getTimeout(configuration).toMillis(), TimeUnit.MILLISECONDS);
    +		AkkaRpcService akkaRpcService = new AkkaRpcService(taskManagerSystem, timeout);
    +
    +		// start high availability service to implement getResourceManagerLeaderRetriever method only
    +		HighAvailabilityServices haServices = new HighAvailabilityServices() {
    +			@Override
    +			public LeaderRetrievalService getResourceManagerLeaderRetriever() throws Exception {
    +				return LeaderRetrievalUtils.createLeaderRetrievalService(configuration);
    +			}
    +
    +			@Override
    +			public LeaderElectionService getJobMasterLeaderElectionService(JobID jobID) throws Exception {
    +				return null;
    +			}
    +		};
    +
    +		// start all the TaskManager services (network stack,  library cache, ...)
    +		// and the TaskManager actor
    +		try {
    +			LOG.info("Starting TaskManager actor");
    +			TaskExecutor taskExecutor = startTaskManagerComponentsAndActor(
    +				configuration,
    +				resourceID,
    +				akkaRpcService,
    +				taskManagerHostname,
    +				haServices,
    +				false);
    +
    +			taskExecutor.start();
    +
    +			// if desired, start the logging daemon that periodically logs the memory usage information
    +			if (LOG.isInfoEnabled() && configuration.getBoolean(
    +				ConfigConstants.TASK_MANAGER_DEBUG_MEMORY_USAGE_START_LOG_THREAD,
    +				ConfigConstants.DEFAULT_TASK_MANAGER_DEBUG_MEMORY_USAGE_START_LOG_THREAD)) {
    +				LOG.info("Starting periodic memory usage logger");
    +
    +				long interval = configuration.getLong(
    +					ConfigConstants.TASK_MANAGER_DEBUG_MEMORY_USAGE_LOG_INTERVAL_MS,
    +					ConfigConstants.DEFAULT_TASK_MANAGER_DEBUG_MEMORY_USAGE_LOG_INTERVAL_MS);
    +
    +				MemoryLogger logger = new MemoryLogger(LOG, interval, taskManagerSystem);
    +				logger.start();
    +			}
    +
    +			// block until everything is done
    +			taskManagerSystem.awaitTermination();
    +		} catch (Throwable t) {
    +			LOG.error("Error while starting up taskManager", t);
    +			try {
    +				taskManagerSystem.shutdown();
    +			} catch (Throwable tt) {
    +				LOG.warn("Could not cleanly shut down actor system", tt);
    +			}
    +			throw t;
    +		}
    +	}
    +
    +	// --------------------------------------------------------------------------
    +	//  Starting and running the TaskManager
    +	// --------------------------------------------------------------------------
    +
    +	/**
    +	 * @param configuration                 The configuration for the TaskManager.
    +	 * @param resourceID                    The id of the resource which the task manager will run on.
    +	 * @param rpcService                  The rpc service which is used to start and connect to the TaskManager RpcEndpoint .
    +	 * @param taskManagerHostname       The hostname/address that describes the TaskManager's data location.
    +	 * @param haServices        Optionally, a high availability service can be provided. If none is given,
    +	 *                                      then a HighAvailabilityServices is constructed from the configuration.
    +	 * @param localTaskManagerCommunication If true, the TaskManager will not initiate the TCP network stack.
    +	 * @return An ActorRef to the TaskManager actor.
    +	 * @throws org.apache.flink.configuration.IllegalConfigurationException     Thrown, if the given config contains illegal values.
    +	 * @throws java.io.IOException      Thrown, if any of the I/O components (such as buffer pools,
    +	 *                                       I/O manager, ...) cannot be properly started.
    +	 * @throws java.lang.Exception      Thrown is some other error occurs while parsing the configuration
    +	 *                                      or starting the TaskManager components.
    +	 */
    +	public static TaskExecutor startTaskManagerComponentsAndActor(
    +		Configuration configuration,
    +		ResourceID resourceID,
    +		RpcService rpcService,
    +		String taskManagerHostname,
    +		HighAvailabilityServices haServices,
    +		boolean localTaskManagerCommunication) throws Exception {
    +
    +		Tuple4<TaskExecutorConfiguration, NetworkEnvironmentConfiguration, InstanceConnectionInfo, MemoryType> tuple4
    +			= parseTaskManagerConfiguration(configuration, taskManagerHostname, localTaskManagerCommunication);
    +
    +		TaskExecutorConfiguration taskExecutorConfig = tuple4.f0;
    +		NetworkEnvironmentConfiguration netConfig = tuple4.f1;
    +		InstanceConnectionInfo connectionInfo = tuple4.f2;
    +		MemoryType memType = tuple4.f3;
    +
    +		// pre-start checks
    +		checkTempDirs(taskExecutorConfig.getTmpDirPaths());
    +
    +		ExecutionContext executionContext = ExecutionContexts$.MODULE$.fromExecutor(new ForkJoinPool());
    +
    +		// we start the network first, to make sure it can allocate its buffers first
    +		NetworkEnvironment network = new NetworkEnvironment(executionContext, taskExecutorConfig.getTimeout(), netConfig);
    +
    +		// computing the amount of memory to use depends on how much memory is available
    +		// it strictly needs to happen AFTER the network stack has been initialized
    +
    +		// check if a value has been configured
    +		long configuredMemory = configuration.getLong(ConfigConstants.TASK_MANAGER_MEMORY_SIZE_KEY, -1L);
    +		checkConfigParameter(configuredMemory == -1 || configuredMemory > 0, configuredMemory,
    +			ConfigConstants.TASK_MANAGER_MEMORY_SIZE_KEY,
    +			"MemoryManager needs at least one MB of memory. " +
    +				"If you leave this config parameter empty, the system automatically " +
    +				"pick a fraction of the available memory.");
    +
    +		long memorySize;
    +		boolean preAllocateMemory = configuration.getBoolean(
    +			ConfigConstants.TASK_MANAGER_MEMORY_PRE_ALLOCATE_KEY,
    +			ConfigConstants.DEFAULT_TASK_MANAGER_MEMORY_PRE_ALLOCATE);
    +		if (configuredMemory > 0) {
    +			if (preAllocateMemory) {
    +				LOG.info("Using {} MB for managed memory." , configuredMemory);
    +			} else {
    +				LOG.info("Limiting managed memory to {} MB, memory will be allocated lazily." , configuredMemory);
    +			}
    +			memorySize = configuredMemory << 20; // megabytes to bytes
    +		} else {
    +			float fraction = configuration.getFloat(
    +				ConfigConstants.TASK_MANAGER_MEMORY_FRACTION_KEY,
    +				ConfigConstants.DEFAULT_MEMORY_MANAGER_MEMORY_FRACTION);
    +			checkConfigParameter(fraction > 0.0f && fraction < 1.0f, fraction,
    +				ConfigConstants.TASK_MANAGER_MEMORY_FRACTION_KEY,
    +				"MemoryManager fraction of the free memory must be between 0.0 and 1.0");
    +
    +			if (memType == MemoryType.HEAP) {
    +				long relativeMemSize = (long) (EnvironmentInformation.getSizeOfFreeHeapMemoryWithDefrag() * fraction);
    +				if (preAllocateMemory) {
    +					LOG.info("Using {} of the currently free heap space for managed heap memory ({} MB)." ,
    +						fraction , relativeMemSize >> 20);
    +				} else {
    +					LOG.info("Limiting managed memory to {} of the currently free heap space ({} MB), " +
    +						"memory will be allocated lazily." , fraction , relativeMemSize >> 20);
    +				}
    +				memorySize = relativeMemSize;
    +			} else if (memType == MemoryType.OFF_HEAP) {
    +				// The maximum heap memory has been adjusted according to the fraction
    +				long maxMemory = EnvironmentInformation.getMaxJvmHeapMemory();
    +				long directMemorySize = (long) (maxMemory / (1.0 - fraction) * fraction);
    +				if (preAllocateMemory) {
    +					LOG.info("Using {} of the maximum memory size for managed off-heap memory ({} MB)." ,
    +						fraction, directMemorySize >> 20);
    +				} else {
    +					LOG.info("Limiting managed memory to {} of the maximum memory size ({} MB)," +
    +						" memory will be allocated lazily.", fraction, directMemorySize >> 20);
    +				}
    +				memorySize = directMemorySize;
    +			} else {
    +				throw new RuntimeException("No supported memory type detected.");
    +			}
    +		}
    +
    +		// now start the memory manager
    +		final MemoryManager memoryManager;
    +		try {
    +			memoryManager = new MemoryManager(
    +				memorySize,
    +				taskExecutorConfig.getNumberOfSlots(),
    +				netConfig.networkBufferSize(),
    +				memType,
    +				preAllocateMemory);
    +		} catch (OutOfMemoryError e) {
    +			if (memType == MemoryType.HEAP) {
    +				throw new Exception("OutOfMemory error (" + e.getMessage() +
    +					") while allocating the TaskManager heap memory (" + memorySize + " bytes).", e);
    +			} else if (memType == MemoryType.OFF_HEAP) {
    +				throw new Exception("OutOfMemory error (" + e.getMessage() +
    +					") while allocating the TaskManager off-heap memory (" + memorySize +
    +					" bytes).Try increasing the maximum direct memory (-XX:MaxDirectMemorySize)", e);
    +			} else {
    +				throw e;
    +			}
    +		}
    +
    +		// start the I/O manager, it will create some temp directories.
    +		IOManager ioManager = new IOManagerAsync(taskExecutorConfig.getTmpDirPaths());
    +
    +		TaskExecutor taskExecutor = new TaskExecutor(
    +			taskExecutorConfig,
    +			resourceID,
    +			connectionInfo,
    +			memoryManager,
    +			ioManager,
    +			network,
    +			taskExecutorConfig.getNumberOfSlots(),
    +			rpcService,
    +			haServices);
    +
    +		return taskExecutor;
    +	}
    +
    +	// --------------------------------------------------------------------------
    +	//  Parsing and checking the TaskManager Configuration
    +	// --------------------------------------------------------------------------
    +
    +	/**
    +	 * Utility method to extract TaskManager config parameters from the configuration and to
    +	 * sanity check them.
    +	 *
    +	 * @param configuration                 The configuration.
    +	 * @param taskManagerHostname           The host name under which the TaskManager communicates.
    +	 * @param localTaskManagerCommunication True, to skip initializing the network stack.
    +	 *                                      Use only in cases where only one task manager runs.
    +	 * @return A tuple (TaskManagerConfiguration, network configuration,
    +	 * InstanceConnectionInfo, JobManager actor Akka URL).
    +	 */
    +	private static Tuple4<TaskExecutorConfiguration, NetworkEnvironmentConfiguration, InstanceConnectionInfo, MemoryType>
    --- End diff --
    
    Could you pack these values into a wrapper? For example, name it `TaskExecutorConfiguration`. It would have the other values as fields.


---
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.
---

[GitHub] flink pull request #2400: [FLINK-4363] Implement TaskManager basic startup o...

Posted by wangzhijiang999 <gi...@git.apache.org>.
Github user wangzhijiang999 commented on a diff in the pull request:

    https://github.com/apache/flink/pull/2400#discussion_r76189450
  
    --- Diff: flink-runtime/src/main/java/org/apache/flink/runtime/rpc/taskexecutor/TaskExecutor.java ---
    @@ -36,27 +82,617 @@
      */
     public class TaskExecutor extends RpcEndpoint<TaskExecutorGateway> {
     
    -	/** The unique resource ID of this TaskExecutor */
    +	private static final Logger LOG = LoggerFactory.getLogger(TaskExecutor.class);
    +
    +	/** The unique resource ID of this TaskManager */
     	private final ResourceID resourceID;
     
     	/** The access to the leader election and metadata storage services */
     	private final HighAvailabilityServices haServices;
     
    -	// --------- resource manager --------
    +	/** The task manager configuration */
    +	private final TaskExecutorConfiguration taskExecutorConfig;
    +
    +	/** The connection information of the task manager */
    +	private final InstanceConnectionInfo connectionInfo;
    +
    +	/** The I/O manager component in the task manager */
    +	private final IOManager ioManager;
    +
    +	/** The memory manager component in the task manager */
    +	private final MemoryManager memoryManager;
    +
    +	/** The network component in the task manager */
    +	private final NetworkEnvironment networkEnvironment;
    +
    +	/** The number of slots in the task manager, should be 1 for YARN */
    +	private final int numberOfSlots;
     
    +	// --------- resource manager --------
     	private TaskExecutorToResourceManagerConnection resourceManagerConnection;
     
     	// ------------------------------------------------------------------------
     
     	public TaskExecutor(
    +			TaskExecutorConfiguration taskExecutorConfig,
    +			ResourceID resourceID,
    +			InstanceConnectionInfo connectionInfo,
    +			MemoryManager memoryManager,
    +			IOManager ioManager,
    +			NetworkEnvironment networkEnvironment,
    +			int numberOfSlots,
     			RpcService rpcService,
    -			HighAvailabilityServices haServices,
    -			ResourceID resourceID) {
    +			HighAvailabilityServices haServices) {
     
     		super(rpcService);
     
    -		this.haServices = checkNotNull(haServices);
    +		this.taskExecutorConfig = checkNotNull(taskExecutorConfig);
     		this.resourceID = checkNotNull(resourceID);
    +		this.connectionInfo = checkNotNull(connectionInfo);
    +		this.memoryManager = checkNotNull(memoryManager);
    +		this.ioManager = checkNotNull(ioManager);
    +		this.networkEnvironment = checkNotNull(networkEnvironment);
    +		this.numberOfSlots = checkNotNull(numberOfSlots);
    +		this.haServices = checkNotNull(haServices);
    +	}
    +
    +	/**
    +	 * Starts and runs the TaskManager.
    +	 * <p/>
    +	 * This method first tries to select the network interface to use for the TaskManager
    +	 * communication. The network interface is used both for the actor communication
    +	 * (coordination) as well as for the data exchange between task managers. Unless
    +	 * the hostname/interface is explicitly configured in the configuration, this
    +	 * method will try out various interfaces and methods to connect to the JobManager
    +	 * and select the one where the connection attempt is successful.
    +	 * <p/>
    +	 * After selecting the network interface, this method brings up an actor system
    +	 * for the TaskManager and its actors, starts the TaskManager's services
    +	 * (library cache, shuffle network stack, ...), and starts the TaskManager itself.
    +	 *
    +	 * @param configuration    The configuration for the TaskManager.
    +	 * @param resourceID       The id of the resource which the task manager will run on.
    +	 */
    +	public static void selectNetworkInterfaceAndRunTaskManager(
    +		Configuration configuration,
    +		ResourceID resourceID) throws Exception {
    +
    +		InetSocketAddress taskManagerAddress = selectNetworkInterfaceAndPort(configuration);
    +
    +		runTaskManager(taskManagerAddress.getHostName(), resourceID, taskManagerAddress.getPort(), configuration);
    +	}
    +
    +	private static InetSocketAddress selectNetworkInterfaceAndPort(Configuration configuration)
    +		throws Exception {
    +		String taskManagerHostname = configuration.getString(ConfigConstants.TASK_MANAGER_HOSTNAME_KEY, null);
    +		if (taskManagerHostname != null) {
    +			LOG.info("Using configured hostname/address for TaskManager: " + taskManagerHostname);
    +		} else {
    +			LeaderRetrievalService leaderRetrievalService = LeaderRetrievalUtils.createLeaderRetrievalService(configuration);
    +			FiniteDuration lookupTimeout = AkkaUtils.getLookupTimeout(configuration);
    +
    +			InetAddress taskManagerAddress = LeaderRetrievalUtils.findConnectingAddress(leaderRetrievalService, lookupTimeout);
    +			taskManagerHostname = taskManagerAddress.getHostName();
    +			LOG.info("TaskManager will use hostname/address '{}' ({}) for communication.",
    +				taskManagerHostname, taskManagerAddress.getHostAddress());
    +		}
    +
    +		// if no task manager port has been configured, use 0 (system will pick any free port)
    +		int actorSystemPort = configuration.getInteger(ConfigConstants.TASK_MANAGER_IPC_PORT_KEY, 0);
    +		if (actorSystemPort < 0 || actorSystemPort > 65535) {
    +			throw new IllegalConfigurationException("Invalid value for '" +
    +				ConfigConstants.TASK_MANAGER_IPC_PORT_KEY +
    +				"' (port for the TaskManager actor system) : " + actorSystemPort +
    +				" - Leave config parameter empty or use 0 to let the system choose a port automatically.");
    +		}
    +
    +		return new InetSocketAddress(taskManagerHostname, actorSystemPort);
    +	}
    +
    +	/**
    +	 * Starts and runs the TaskManager. Brings up an actor system for the TaskManager and its
    +	 * actors, starts the TaskManager's services (library cache, shuffle network stack, ...),
    +	 * and starts the TaskManager itself.
    +	 * <p/>
    +	 * This method will also spawn a process reaper for the TaskManager (kill the process if
    +	 * the actor fails) and optionally start the JVM memory logging thread.
    +	 *
    +	 * @param taskManagerHostname The hostname/address of the interface where the actor system
    +	 *                            will communicate.
    +	 * @param resourceID          The id of the resource which the task manager will run on.
    +	 * @param actorSystemPort   The port at which the actor system will communicate.
    +	 * @param configuration       The configuration for the TaskManager.
    +	 */
    +	private static void runTaskManager(
    +		String taskManagerHostname,
    +		ResourceID resourceID,
    +		int actorSystemPort,
    +		final Configuration configuration) throws Exception {
    +
    +		LOG.info("Starting TaskManager");
    +
    +		// Bring up the TaskManager actor system first, bind it to the given address.
    +
    +		LOG.info("Starting TaskManager actor system at " +
    +			NetUtils.hostAndPortToUrlString(taskManagerHostname, actorSystemPort));
    +
    +		final ActorSystem taskManagerSystem;
    +		try {
    +			Tuple2<String, Object> address = new Tuple2<String, Object>(taskManagerHostname, actorSystemPort);
    +			Config akkaConfig = AkkaUtils.getAkkaConfig(configuration, new Some<>(address));
    +			LOG.debug("Using akka configuration\n " + akkaConfig);
    +			taskManagerSystem = AkkaUtils.createActorSystem(akkaConfig);
    +		} catch (Throwable t) {
    +			if (t instanceof org.jboss.netty.channel.ChannelException) {
    +				Throwable cause = t.getCause();
    +				if (cause != null && t.getCause() instanceof java.net.BindException) {
    +					String address = NetUtils.hostAndPortToUrlString(taskManagerHostname, actorSystemPort);
    +					throw new IOException("Unable to bind TaskManager actor system to address " +
    +						address + " - " + cause.getMessage(), t);
    +				}
    +			}
    +			throw new Exception("Could not create TaskManager actor system", t);
    +		}
    +
    +		// start akka rpc service based on actor system
    +		Timeout timeout = new Timeout(AkkaUtils.getTimeout(configuration).toMillis(), TimeUnit.MILLISECONDS);
    +		AkkaRpcService akkaRpcService = new AkkaRpcService(taskManagerSystem, timeout);
    +
    +		// start high availability service to implement getResourceManagerLeaderRetriever method only
    +		HighAvailabilityServices haServices = new HighAvailabilityServices() {
    +			@Override
    +			public LeaderRetrievalService getResourceManagerLeaderRetriever() throws Exception {
    +				return LeaderRetrievalUtils.createLeaderRetrievalService(configuration);
    +			}
    +
    +			@Override
    +			public LeaderElectionService getJobMasterLeaderElectionService(JobID jobID) throws Exception {
    +				return null;
    +			}
    +		};
    +
    +		// start all the TaskManager services (network stack,  library cache, ...)
    +		// and the TaskManager actor
    +		try {
    +			LOG.info("Starting TaskManager actor");
    +			TaskExecutor taskExecutor = startTaskManagerComponentsAndActor(
    +				configuration,
    +				resourceID,
    +				akkaRpcService,
    +				taskManagerHostname,
    +				haServices,
    +				false);
    +
    +			taskExecutor.start();
    +
    +			// if desired, start the logging daemon that periodically logs the memory usage information
    +			if (LOG.isInfoEnabled() && configuration.getBoolean(
    +				ConfigConstants.TASK_MANAGER_DEBUG_MEMORY_USAGE_START_LOG_THREAD,
    +				ConfigConstants.DEFAULT_TASK_MANAGER_DEBUG_MEMORY_USAGE_START_LOG_THREAD)) {
    +				LOG.info("Starting periodic memory usage logger");
    +
    +				long interval = configuration.getLong(
    +					ConfigConstants.TASK_MANAGER_DEBUG_MEMORY_USAGE_LOG_INTERVAL_MS,
    +					ConfigConstants.DEFAULT_TASK_MANAGER_DEBUG_MEMORY_USAGE_LOG_INTERVAL_MS);
    +
    +				MemoryLogger logger = new MemoryLogger(LOG, interval, taskManagerSystem);
    +				logger.start();
    +			}
    +
    +			// block until everything is done
    +			taskManagerSystem.awaitTermination();
    +		} catch (Throwable t) {
    +			LOG.error("Error while starting up taskManager", t);
    +			try {
    +				taskManagerSystem.shutdown();
    +			} catch (Throwable tt) {
    +				LOG.warn("Could not cleanly shut down actor system", tt);
    +			}
    +			throw t;
    +		}
    +	}
    +
    +	// --------------------------------------------------------------------------
    +	//  Starting and running the TaskManager
    +	// --------------------------------------------------------------------------
    +
    +	/**
    +	 * @param configuration                 The configuration for the TaskManager.
    +	 * @param resourceID                    The id of the resource which the task manager will run on.
    +	 * @param rpcService                  The rpc service which is used to start and connect to the TaskManager RpcEndpoint .
    +	 * @param taskManagerHostname       The hostname/address that describes the TaskManager's data location.
    +	 * @param haServices        Optionally, a high availability service can be provided. If none is given,
    +	 *                                      then a HighAvailabilityServices is constructed from the configuration.
    +	 * @param localTaskManagerCommunication If true, the TaskManager will not initiate the TCP network stack.
    +	 * @return An ActorRef to the TaskManager actor.
    +	 * @throws org.apache.flink.configuration.IllegalConfigurationException     Thrown, if the given config contains illegal values.
    +	 * @throws java.io.IOException      Thrown, if any of the I/O components (such as buffer pools,
    +	 *                                       I/O manager, ...) cannot be properly started.
    +	 * @throws java.lang.Exception      Thrown is some other error occurs while parsing the configuration
    +	 *                                      or starting the TaskManager components.
    +	 */
    +	public static TaskExecutor startTaskManagerComponentsAndActor(
    +		Configuration configuration,
    +		ResourceID resourceID,
    +		RpcService rpcService,
    +		String taskManagerHostname,
    +		HighAvailabilityServices haServices,
    +		boolean localTaskManagerCommunication) throws Exception {
    +
    +		Tuple4<TaskExecutorConfiguration, NetworkEnvironmentConfiguration, InstanceConnectionInfo, MemoryType> tuple4
    +			= parseTaskManagerConfiguration(configuration, taskManagerHostname, localTaskManagerCommunication);
    +
    +		TaskExecutorConfiguration taskExecutorConfig = tuple4.f0;
    +		NetworkEnvironmentConfiguration netConfig = tuple4.f1;
    +		InstanceConnectionInfo connectionInfo = tuple4.f2;
    +		MemoryType memType = tuple4.f3;
    +
    +		// pre-start checks
    +		checkTempDirs(taskExecutorConfig.getTmpDirPaths());
    +
    +		ExecutionContext executionContext = ExecutionContexts$.MODULE$.fromExecutor(new ForkJoinPool());
    +
    +		// we start the network first, to make sure it can allocate its buffers first
    +		NetworkEnvironment network = new NetworkEnvironment(executionContext, taskExecutorConfig.getTimeout(), netConfig);
    +
    +		// computing the amount of memory to use depends on how much memory is available
    +		// it strictly needs to happen AFTER the network stack has been initialized
    +
    +		// check if a value has been configured
    +		long configuredMemory = configuration.getLong(ConfigConstants.TASK_MANAGER_MEMORY_SIZE_KEY, -1L);
    +		checkConfigParameter(configuredMemory == -1 || configuredMemory > 0, configuredMemory,
    +			ConfigConstants.TASK_MANAGER_MEMORY_SIZE_KEY,
    +			"MemoryManager needs at least one MB of memory. " +
    +				"If you leave this config parameter empty, the system automatically " +
    +				"pick a fraction of the available memory.");
    +
    +		long memorySize;
    +		boolean preAllocateMemory = configuration.getBoolean(
    +			ConfigConstants.TASK_MANAGER_MEMORY_PRE_ALLOCATE_KEY,
    +			ConfigConstants.DEFAULT_TASK_MANAGER_MEMORY_PRE_ALLOCATE);
    +		if (configuredMemory > 0) {
    +			if (preAllocateMemory) {
    +				LOG.info("Using {} MB for managed memory." , configuredMemory);
    +			} else {
    +				LOG.info("Limiting managed memory to {} MB, memory will be allocated lazily." , configuredMemory);
    +			}
    +			memorySize = configuredMemory << 20; // megabytes to bytes
    +		} else {
    +			float fraction = configuration.getFloat(
    +				ConfigConstants.TASK_MANAGER_MEMORY_FRACTION_KEY,
    +				ConfigConstants.DEFAULT_MEMORY_MANAGER_MEMORY_FRACTION);
    +			checkConfigParameter(fraction > 0.0f && fraction < 1.0f, fraction,
    +				ConfigConstants.TASK_MANAGER_MEMORY_FRACTION_KEY,
    +				"MemoryManager fraction of the free memory must be between 0.0 and 1.0");
    +
    +			if (memType == MemoryType.HEAP) {
    +				long relativeMemSize = (long) (EnvironmentInformation.getSizeOfFreeHeapMemoryWithDefrag() * fraction);
    +				if (preAllocateMemory) {
    +					LOG.info("Using {} of the currently free heap space for managed heap memory ({} MB)." ,
    +						fraction , relativeMemSize >> 20);
    +				} else {
    +					LOG.info("Limiting managed memory to {} of the currently free heap space ({} MB), " +
    +						"memory will be allocated lazily." , fraction , relativeMemSize >> 20);
    +				}
    +				memorySize = relativeMemSize;
    +			} else if (memType == MemoryType.OFF_HEAP) {
    +				// The maximum heap memory has been adjusted according to the fraction
    +				long maxMemory = EnvironmentInformation.getMaxJvmHeapMemory();
    +				long directMemorySize = (long) (maxMemory / (1.0 - fraction) * fraction);
    +				if (preAllocateMemory) {
    +					LOG.info("Using {} of the maximum memory size for managed off-heap memory ({} MB)." ,
    +						fraction, directMemorySize >> 20);
    +				} else {
    +					LOG.info("Limiting managed memory to {} of the maximum memory size ({} MB)," +
    +						" memory will be allocated lazily.", fraction, directMemorySize >> 20);
    +				}
    +				memorySize = directMemorySize;
    +			} else {
    +				throw new RuntimeException("No supported memory type detected.");
    +			}
    +		}
    +
    +		// now start the memory manager
    +		final MemoryManager memoryManager;
    +		try {
    +			memoryManager = new MemoryManager(
    +				memorySize,
    +				taskExecutorConfig.getNumberOfSlots(),
    +				netConfig.networkBufferSize(),
    +				memType,
    +				preAllocateMemory);
    +		} catch (OutOfMemoryError e) {
    +			if (memType == MemoryType.HEAP) {
    +				throw new Exception("OutOfMemory error (" + e.getMessage() +
    +					") while allocating the TaskManager heap memory (" + memorySize + " bytes).", e);
    +			} else if (memType == MemoryType.OFF_HEAP) {
    +				throw new Exception("OutOfMemory error (" + e.getMessage() +
    +					") while allocating the TaskManager off-heap memory (" + memorySize +
    +					" bytes).Try increasing the maximum direct memory (-XX:MaxDirectMemorySize)", e);
    +			} else {
    +				throw e;
    +			}
    +		}
    +
    +		// start the I/O manager, it will create some temp directories.
    +		IOManager ioManager = new IOManagerAsync(taskExecutorConfig.getTmpDirPaths());
    +
    +		TaskExecutor taskExecutor = new TaskExecutor(
    +			taskExecutorConfig,
    +			resourceID,
    +			connectionInfo,
    +			memoryManager,
    +			ioManager,
    +			network,
    +			taskExecutorConfig.getNumberOfSlots(),
    +			rpcService,
    +			haServices);
    +
    +		return taskExecutor;
    +	}
    +
    +	// --------------------------------------------------------------------------
    +	//  Parsing and checking the TaskManager Configuration
    +	// --------------------------------------------------------------------------
    +
    +	/**
    +	 * Utility method to extract TaskManager config parameters from the configuration and to
    +	 * sanity check them.
    +	 *
    +	 * @param configuration                 The configuration.
    +	 * @param taskManagerHostname           The host name under which the TaskManager communicates.
    +	 * @param localTaskManagerCommunication True, to skip initializing the network stack.
    +	 *                                      Use only in cases where only one task manager runs.
    +	 * @return A tuple (TaskManagerConfiguration, network configuration,
    +	 * InstanceConnectionInfo, JobManager actor Akka URL).
    +	 */
    +	private static Tuple4<TaskExecutorConfiguration, NetworkEnvironmentConfiguration, InstanceConnectionInfo, MemoryType>
    --- End diff --
    
    The tuple4 has been replaced  and all the values are put into the TaskExecutorConfiguration as fields.


---
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.
---

[GitHub] flink issue #2400: [FLINK-4363] Implement TaskManager basic startup of all c...

Posted by mxm <gi...@git.apache.org>.
Github user mxm commented on the issue:

    https://github.com/apache/flink/pull/2400
  
    Hi @wangzhijiang999! `ActorSystem` and `LeaderRetrievalService` should only every appear in the factory (startTaskManagerComponentsAndActor) which bring up the TaskExecutor component. The TaskExecutor itself uses the new classes `RpcService` and `HighAvailabilityService`. All other code should change much, except for the Scala->Java changes.


---
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.
---

[GitHub] flink pull request #2400: [FLINK-4363] Implement TaskManager basic startup o...

Posted by wangzhijiang999 <gi...@git.apache.org>.
Github user wangzhijiang999 commented on a diff in the pull request:

    https://github.com/apache/flink/pull/2400#discussion_r76189882
  
    --- Diff: flink-runtime/src/main/java/org/apache/flink/runtime/rpc/taskexecutor/TaskExecutor.java ---
    @@ -36,27 +82,617 @@
      */
     public class TaskExecutor extends RpcEndpoint<TaskExecutorGateway> {
     
    -	/** The unique resource ID of this TaskExecutor */
    +	private static final Logger LOG = LoggerFactory.getLogger(TaskExecutor.class);
    +
    +	/** The unique resource ID of this TaskManager */
     	private final ResourceID resourceID;
     
     	/** The access to the leader election and metadata storage services */
     	private final HighAvailabilityServices haServices;
     
    -	// --------- resource manager --------
    +	/** The task manager configuration */
    +	private final TaskExecutorConfiguration taskExecutorConfig;
    +
    +	/** The connection information of the task manager */
    +	private final InstanceConnectionInfo connectionInfo;
    +
    +	/** The I/O manager component in the task manager */
    +	private final IOManager ioManager;
    +
    +	/** The memory manager component in the task manager */
    +	private final MemoryManager memoryManager;
    +
    +	/** The network component in the task manager */
    +	private final NetworkEnvironment networkEnvironment;
    +
    +	/** The number of slots in the task manager, should be 1 for YARN */
    +	private final int numberOfSlots;
     
    +	// --------- resource manager --------
     	private TaskExecutorToResourceManagerConnection resourceManagerConnection;
     
     	// ------------------------------------------------------------------------
     
     	public TaskExecutor(
    +			TaskExecutorConfiguration taskExecutorConfig,
    +			ResourceID resourceID,
    +			InstanceConnectionInfo connectionInfo,
    +			MemoryManager memoryManager,
    +			IOManager ioManager,
    +			NetworkEnvironment networkEnvironment,
    +			int numberOfSlots,
     			RpcService rpcService,
    -			HighAvailabilityServices haServices,
    -			ResourceID resourceID) {
    +			HighAvailabilityServices haServices) {
     
     		super(rpcService);
     
    -		this.haServices = checkNotNull(haServices);
    +		this.taskExecutorConfig = checkNotNull(taskExecutorConfig);
     		this.resourceID = checkNotNull(resourceID);
    +		this.connectionInfo = checkNotNull(connectionInfo);
    +		this.memoryManager = checkNotNull(memoryManager);
    +		this.ioManager = checkNotNull(ioManager);
    +		this.networkEnvironment = checkNotNull(networkEnvironment);
    +		this.numberOfSlots = checkNotNull(numberOfSlots);
    +		this.haServices = checkNotNull(haServices);
    +	}
    +
    +	/**
    +	 * Starts and runs the TaskManager.
    +	 * <p/>
    +	 * This method first tries to select the network interface to use for the TaskManager
    +	 * communication. The network interface is used both for the actor communication
    +	 * (coordination) as well as for the data exchange between task managers. Unless
    +	 * the hostname/interface is explicitly configured in the configuration, this
    +	 * method will try out various interfaces and methods to connect to the JobManager
    +	 * and select the one where the connection attempt is successful.
    +	 * <p/>
    +	 * After selecting the network interface, this method brings up an actor system
    +	 * for the TaskManager and its actors, starts the TaskManager's services
    +	 * (library cache, shuffle network stack, ...), and starts the TaskManager itself.
    +	 *
    +	 * @param configuration    The configuration for the TaskManager.
    +	 * @param resourceID       The id of the resource which the task manager will run on.
    +	 */
    +	public static void selectNetworkInterfaceAndRunTaskManager(
    +		Configuration configuration,
    +		ResourceID resourceID) throws Exception {
    +
    +		InetSocketAddress taskManagerAddress = selectNetworkInterfaceAndPort(configuration);
    +
    +		runTaskManager(taskManagerAddress.getHostName(), resourceID, taskManagerAddress.getPort(), configuration);
    +	}
    +
    +	private static InetSocketAddress selectNetworkInterfaceAndPort(Configuration configuration)
    +		throws Exception {
    +		String taskManagerHostname = configuration.getString(ConfigConstants.TASK_MANAGER_HOSTNAME_KEY, null);
    +		if (taskManagerHostname != null) {
    +			LOG.info("Using configured hostname/address for TaskManager: " + taskManagerHostname);
    +		} else {
    +			LeaderRetrievalService leaderRetrievalService = LeaderRetrievalUtils.createLeaderRetrievalService(configuration);
    +			FiniteDuration lookupTimeout = AkkaUtils.getLookupTimeout(configuration);
    +
    +			InetAddress taskManagerAddress = LeaderRetrievalUtils.findConnectingAddress(leaderRetrievalService, lookupTimeout);
    +			taskManagerHostname = taskManagerAddress.getHostName();
    +			LOG.info("TaskManager will use hostname/address '{}' ({}) for communication.",
    +				taskManagerHostname, taskManagerAddress.getHostAddress());
    +		}
    +
    +		// if no task manager port has been configured, use 0 (system will pick any free port)
    +		int actorSystemPort = configuration.getInteger(ConfigConstants.TASK_MANAGER_IPC_PORT_KEY, 0);
    +		if (actorSystemPort < 0 || actorSystemPort > 65535) {
    +			throw new IllegalConfigurationException("Invalid value for '" +
    +				ConfigConstants.TASK_MANAGER_IPC_PORT_KEY +
    +				"' (port for the TaskManager actor system) : " + actorSystemPort +
    +				" - Leave config parameter empty or use 0 to let the system choose a port automatically.");
    +		}
    +
    +		return new InetSocketAddress(taskManagerHostname, actorSystemPort);
    +	}
    +
    +	/**
    +	 * Starts and runs the TaskManager. Brings up an actor system for the TaskManager and its
    +	 * actors, starts the TaskManager's services (library cache, shuffle network stack, ...),
    +	 * and starts the TaskManager itself.
    +	 * <p/>
    +	 * This method will also spawn a process reaper for the TaskManager (kill the process if
    +	 * the actor fails) and optionally start the JVM memory logging thread.
    +	 *
    +	 * @param taskManagerHostname The hostname/address of the interface where the actor system
    +	 *                            will communicate.
    +	 * @param resourceID          The id of the resource which the task manager will run on.
    +	 * @param actorSystemPort   The port at which the actor system will communicate.
    +	 * @param configuration       The configuration for the TaskManager.
    +	 */
    +	private static void runTaskManager(
    +		String taskManagerHostname,
    +		ResourceID resourceID,
    +		int actorSystemPort,
    +		final Configuration configuration) throws Exception {
    +
    +		LOG.info("Starting TaskManager");
    +
    +		// Bring up the TaskManager actor system first, bind it to the given address.
    +
    +		LOG.info("Starting TaskManager actor system at " +
    +			NetUtils.hostAndPortToUrlString(taskManagerHostname, actorSystemPort));
    +
    +		final ActorSystem taskManagerSystem;
    +		try {
    +			Tuple2<String, Object> address = new Tuple2<String, Object>(taskManagerHostname, actorSystemPort);
    +			Config akkaConfig = AkkaUtils.getAkkaConfig(configuration, new Some<>(address));
    +			LOG.debug("Using akka configuration\n " + akkaConfig);
    +			taskManagerSystem = AkkaUtils.createActorSystem(akkaConfig);
    +		} catch (Throwable t) {
    +			if (t instanceof org.jboss.netty.channel.ChannelException) {
    +				Throwable cause = t.getCause();
    +				if (cause != null && t.getCause() instanceof java.net.BindException) {
    +					String address = NetUtils.hostAndPortToUrlString(taskManagerHostname, actorSystemPort);
    +					throw new IOException("Unable to bind TaskManager actor system to address " +
    +						address + " - " + cause.getMessage(), t);
    +				}
    +			}
    +			throw new Exception("Could not create TaskManager actor system", t);
    +		}
    +
    +		// start akka rpc service based on actor system
    +		Timeout timeout = new Timeout(AkkaUtils.getTimeout(configuration).toMillis(), TimeUnit.MILLISECONDS);
    +		AkkaRpcService akkaRpcService = new AkkaRpcService(taskManagerSystem, timeout);
    +
    +		// start high availability service to implement getResourceManagerLeaderRetriever method only
    +		HighAvailabilityServices haServices = new HighAvailabilityServices() {
    +			@Override
    +			public LeaderRetrievalService getResourceManagerLeaderRetriever() throws Exception {
    +				return LeaderRetrievalUtils.createLeaderRetrievalService(configuration);
    +			}
    +
    +			@Override
    +			public LeaderElectionService getJobMasterLeaderElectionService(JobID jobID) throws Exception {
    +				return null;
    +			}
    +		};
    +
    +		// start all the TaskManager services (network stack,  library cache, ...)
    +		// and the TaskManager actor
    +		try {
    +			LOG.info("Starting TaskManager actor");
    +			TaskExecutor taskExecutor = startTaskManagerComponentsAndActor(
    +				configuration,
    +				resourceID,
    +				akkaRpcService,
    +				taskManagerHostname,
    +				haServices,
    +				false);
    +
    +			taskExecutor.start();
    +
    +			// if desired, start the logging daemon that periodically logs the memory usage information
    +			if (LOG.isInfoEnabled() && configuration.getBoolean(
    +				ConfigConstants.TASK_MANAGER_DEBUG_MEMORY_USAGE_START_LOG_THREAD,
    +				ConfigConstants.DEFAULT_TASK_MANAGER_DEBUG_MEMORY_USAGE_START_LOG_THREAD)) {
    +				LOG.info("Starting periodic memory usage logger");
    +
    +				long interval = configuration.getLong(
    +					ConfigConstants.TASK_MANAGER_DEBUG_MEMORY_USAGE_LOG_INTERVAL_MS,
    +					ConfigConstants.DEFAULT_TASK_MANAGER_DEBUG_MEMORY_USAGE_LOG_INTERVAL_MS);
    +
    +				MemoryLogger logger = new MemoryLogger(LOG, interval, taskManagerSystem);
    +				logger.start();
    +			}
    +
    +			// block until everything is done
    +			taskManagerSystem.awaitTermination();
    +		} catch (Throwable t) {
    +			LOG.error("Error while starting up taskManager", t);
    +			try {
    +				taskManagerSystem.shutdown();
    +			} catch (Throwable tt) {
    +				LOG.warn("Could not cleanly shut down actor system", tt);
    +			}
    +			throw t;
    +		}
    +	}
    +
    +	// --------------------------------------------------------------------------
    +	//  Starting and running the TaskManager
    +	// --------------------------------------------------------------------------
    +
    +	/**
    +	 * @param configuration                 The configuration for the TaskManager.
    +	 * @param resourceID                    The id of the resource which the task manager will run on.
    +	 * @param rpcService                  The rpc service which is used to start and connect to the TaskManager RpcEndpoint .
    +	 * @param taskManagerHostname       The hostname/address that describes the TaskManager's data location.
    +	 * @param haServices        Optionally, a high availability service can be provided. If none is given,
    +	 *                                      then a HighAvailabilityServices is constructed from the configuration.
    +	 * @param localTaskManagerCommunication If true, the TaskManager will not initiate the TCP network stack.
    +	 * @return An ActorRef to the TaskManager actor.
    +	 * @throws org.apache.flink.configuration.IllegalConfigurationException     Thrown, if the given config contains illegal values.
    +	 * @throws java.io.IOException      Thrown, if any of the I/O components (such as buffer pools,
    +	 *                                       I/O manager, ...) cannot be properly started.
    +	 * @throws java.lang.Exception      Thrown is some other error occurs while parsing the configuration
    +	 *                                      or starting the TaskManager components.
    +	 */
    +	public static TaskExecutor startTaskManagerComponentsAndActor(
    +		Configuration configuration,
    +		ResourceID resourceID,
    +		RpcService rpcService,
    +		String taskManagerHostname,
    +		HighAvailabilityServices haServices,
    +		boolean localTaskManagerCommunication) throws Exception {
    +
    +		Tuple4<TaskExecutorConfiguration, NetworkEnvironmentConfiguration, InstanceConnectionInfo, MemoryType> tuple4
    +			= parseTaskManagerConfiguration(configuration, taskManagerHostname, localTaskManagerCommunication);
    +
    +		TaskExecutorConfiguration taskExecutorConfig = tuple4.f0;
    +		NetworkEnvironmentConfiguration netConfig = tuple4.f1;
    +		InstanceConnectionInfo connectionInfo = tuple4.f2;
    +		MemoryType memType = tuple4.f3;
    +
    +		// pre-start checks
    +		checkTempDirs(taskExecutorConfig.getTmpDirPaths());
    +
    +		ExecutionContext executionContext = ExecutionContexts$.MODULE$.fromExecutor(new ForkJoinPool());
    +
    +		// we start the network first, to make sure it can allocate its buffers first
    +		NetworkEnvironment network = new NetworkEnvironment(executionContext, taskExecutorConfig.getTimeout(), netConfig);
    +
    +		// computing the amount of memory to use depends on how much memory is available
    +		// it strictly needs to happen AFTER the network stack has been initialized
    +
    +		// check if a value has been configured
    +		long configuredMemory = configuration.getLong(ConfigConstants.TASK_MANAGER_MEMORY_SIZE_KEY, -1L);
    +		checkConfigParameter(configuredMemory == -1 || configuredMemory > 0, configuredMemory,
    +			ConfigConstants.TASK_MANAGER_MEMORY_SIZE_KEY,
    +			"MemoryManager needs at least one MB of memory. " +
    +				"If you leave this config parameter empty, the system automatically " +
    +				"pick a fraction of the available memory.");
    +
    +		long memorySize;
    +		boolean preAllocateMemory = configuration.getBoolean(
    +			ConfigConstants.TASK_MANAGER_MEMORY_PRE_ALLOCATE_KEY,
    +			ConfigConstants.DEFAULT_TASK_MANAGER_MEMORY_PRE_ALLOCATE);
    +		if (configuredMemory > 0) {
    +			if (preAllocateMemory) {
    +				LOG.info("Using {} MB for managed memory." , configuredMemory);
    +			} else {
    +				LOG.info("Limiting managed memory to {} MB, memory will be allocated lazily." , configuredMemory);
    +			}
    +			memorySize = configuredMemory << 20; // megabytes to bytes
    +		} else {
    +			float fraction = configuration.getFloat(
    +				ConfigConstants.TASK_MANAGER_MEMORY_FRACTION_KEY,
    +				ConfigConstants.DEFAULT_MEMORY_MANAGER_MEMORY_FRACTION);
    +			checkConfigParameter(fraction > 0.0f && fraction < 1.0f, fraction,
    +				ConfigConstants.TASK_MANAGER_MEMORY_FRACTION_KEY,
    +				"MemoryManager fraction of the free memory must be between 0.0 and 1.0");
    +
    +			if (memType == MemoryType.HEAP) {
    +				long relativeMemSize = (long) (EnvironmentInformation.getSizeOfFreeHeapMemoryWithDefrag() * fraction);
    +				if (preAllocateMemory) {
    +					LOG.info("Using {} of the currently free heap space for managed heap memory ({} MB)." ,
    +						fraction , relativeMemSize >> 20);
    +				} else {
    +					LOG.info("Limiting managed memory to {} of the currently free heap space ({} MB), " +
    +						"memory will be allocated lazily." , fraction , relativeMemSize >> 20);
    +				}
    +				memorySize = relativeMemSize;
    +			} else if (memType == MemoryType.OFF_HEAP) {
    +				// The maximum heap memory has been adjusted according to the fraction
    +				long maxMemory = EnvironmentInformation.getMaxJvmHeapMemory();
    +				long directMemorySize = (long) (maxMemory / (1.0 - fraction) * fraction);
    +				if (preAllocateMemory) {
    +					LOG.info("Using {} of the maximum memory size for managed off-heap memory ({} MB)." ,
    +						fraction, directMemorySize >> 20);
    +				} else {
    +					LOG.info("Limiting managed memory to {} of the maximum memory size ({} MB)," +
    +						" memory will be allocated lazily.", fraction, directMemorySize >> 20);
    +				}
    +				memorySize = directMemorySize;
    +			} else {
    +				throw new RuntimeException("No supported memory type detected.");
    +			}
    +		}
    +
    +		// now start the memory manager
    +		final MemoryManager memoryManager;
    +		try {
    +			memoryManager = new MemoryManager(
    +				memorySize,
    +				taskExecutorConfig.getNumberOfSlots(),
    +				netConfig.networkBufferSize(),
    +				memType,
    +				preAllocateMemory);
    +		} catch (OutOfMemoryError e) {
    +			if (memType == MemoryType.HEAP) {
    +				throw new Exception("OutOfMemory error (" + e.getMessage() +
    +					") while allocating the TaskManager heap memory (" + memorySize + " bytes).", e);
    +			} else if (memType == MemoryType.OFF_HEAP) {
    +				throw new Exception("OutOfMemory error (" + e.getMessage() +
    +					") while allocating the TaskManager off-heap memory (" + memorySize +
    +					" bytes).Try increasing the maximum direct memory (-XX:MaxDirectMemorySize)", e);
    +			} else {
    +				throw e;
    +			}
    +		}
    +
    +		// start the I/O manager, it will create some temp directories.
    +		IOManager ioManager = new IOManagerAsync(taskExecutorConfig.getTmpDirPaths());
    +
    +		TaskExecutor taskExecutor = new TaskExecutor(
    +			taskExecutorConfig,
    +			resourceID,
    +			connectionInfo,
    +			memoryManager,
    +			ioManager,
    +			network,
    +			taskExecutorConfig.getNumberOfSlots(),
    +			rpcService,
    +			haServices);
    +
    +		return taskExecutor;
    +	}
    +
    +	// --------------------------------------------------------------------------
    +	//  Parsing and checking the TaskManager Configuration
    +	// --------------------------------------------------------------------------
    +
    +	/**
    +	 * Utility method to extract TaskManager config parameters from the configuration and to
    +	 * sanity check them.
    +	 *
    +	 * @param configuration                 The configuration.
    +	 * @param taskManagerHostname           The host name under which the TaskManager communicates.
    +	 * @param localTaskManagerCommunication True, to skip initializing the network stack.
    +	 *                                      Use only in cases where only one task manager runs.
    +	 * @return A tuple (TaskManagerConfiguration, network configuration,
    +	 * InstanceConnectionInfo, JobManager actor Akka URL).
    +	 */
    +	private static Tuple4<TaskExecutorConfiguration, NetworkEnvironmentConfiguration, InstanceConnectionInfo, MemoryType>
    --- End diff --
    
    For the factory issue, as I understand,  add a new TaskExecutorFactory class and remove "selectNetworkInterfaceAndRunTaskManager" , "startTaskManagerComponentsAndActor" related methods into TaskExecutorFactory from TaskExecutor class. So the outsider world can invoke factory methods to construct and run TaskExecutor instance. Is it right?


---
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.
---

[GitHub] flink pull request #2400: [FLINK-4363] Implement TaskManager basic startup o...

Posted by wangzhijiang999 <gi...@git.apache.org>.
Github user wangzhijiang999 closed the pull request at:

    https://github.com/apache/flink/pull/2400


---
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.
---

[GitHub] flink pull request #2400: [FLINK-4363] Implement TaskManager basic startup o...

Posted by wangzhijiang999 <gi...@git.apache.org>.
Github user wangzhijiang999 commented on a diff in the pull request:

    https://github.com/apache/flink/pull/2400#discussion_r75805009
  
    --- Diff: flink-runtime/src/main/java/org/apache/flink/runtime/rpc/taskexecutor/TaskExecutor.java ---
    @@ -35,27 +79,634 @@
      */
     public class TaskExecutor extends RpcEndpoint<TaskExecutorGateway> {
     
    -	/** The unique resource ID of this TaskExecutor */
    +	private static final Logger LOG = LoggerFactory.getLogger(TaskExecutor.class);
    +
    +	/** Return code for critical errors during the runtime */
    +	private static final int RUNTIME_FAILURE_RETURN_CODE = 2;
    +
    +	/** The name of the TaskManager actor */
    +	private static final String TASK_MANAGER_NAME = "taskmanager";
    +
    +	/** The unique resource ID of this TaskManager */
     	private final ResourceID resourceID;
     
     	/** The access to the leader election and metadata storage services */
     	private final HighAvailabilityServices haServices;
     
    -	// --------- resource manager --------
    +	/** The task manager configuration */
    +	private final TaskManagerConfiguration taskManagerConfig;
    +
    +	/** The connection information of the task manager */
    +	private final InstanceConnectionInfo connectionInfo;
    +
    +	/** The I/O manager component in the task manager */
    +	private final IOManager ioManager;
    +
    +	/** The memory manager component in the task manager */
    +	private final MemoryManager memoryManager;
    +
    +	/** The network component in the task manager */
    +	private final NetworkEnvironment networkEnvironment;
    +
    +	/** The number of slots in the task manager, should be 1 for YARN */
    +	private final int numberOfSlots;
     
    +	// --------- resource manager --------
     	private TaskExecutorToResourceManagerConnection resourceManagerConnection;
     
     	// ------------------------------------------------------------------------
     
     	public TaskExecutor(
    +			TaskManagerConfiguration taskManagerConfig,
    +			ResourceID resourceID,
    +			InstanceConnectionInfo connectionInfo,
    +			MemoryManager memoryManager,
    +			IOManager ioManager,
    +			NetworkEnvironment networkEnvironment,
    +			int numberOfSlots,
     			RpcService rpcService,
    -			HighAvailabilityServices haServices,
    -			ResourceID resourceID) {
    +			HighAvailabilityServices haServices) {
     
     		super(rpcService);
     
    -		this.haServices = checkNotNull(haServices);
    +		this.taskManagerConfig = checkNotNull(taskManagerConfig);
     		this.resourceID = checkNotNull(resourceID);
    +		this.connectionInfo = checkNotNull(connectionInfo);
    +		this.memoryManager = checkNotNull(memoryManager);
    +		this.ioManager = checkNotNull(ioManager);
    +		this.networkEnvironment = checkNotNull(networkEnvironment);
    +		this.numberOfSlots = checkNotNull(numberOfSlots);
    +		this.haServices = checkNotNull(haServices);
    +	}
    +
    +	/**
    +	 * Starts and runs the TaskManager.
    +	 * <p/>
    +	 * This method first tries to select the network interface to use for the TaskManager
    +	 * communication. The network interface is used both for the actor communication
    +	 * (coordination) as well as for the data exchange between task managers. Unless
    +	 * the hostname/interface is explicitly configured in the configuration, this
    +	 * method will try out various interfaces and methods to connect to the JobManager
    +	 * and select the one where the connection attempt is successful.
    +	 * <p/>
    +	 * After selecting the network interface, this method brings up an actor system
    +	 * for the TaskManager and its actors, starts the TaskManager's services
    +	 * (library cache, shuffle network stack, ...), and starts the TaskManager itself.
    +	 *
    +	 * @param configuration    The configuration for the TaskManager.
    +	 * @param taskManagerClass The actor class to instantiate.
    +	 *                         Allows to use TaskManager subclasses for example for YARN.
    +	 */
    +	public static void selectNetworkInterfaceAndRunTaskManager(
    +		Configuration configuration,
    +		ResourceID resourceID,
    +		Class<? extends TaskManager> taskManagerClass) throws Exception {
    +
    +		Tuple2<String, Integer> tuple2 = selectNetworkInterfaceAndPort(configuration);
    +
    +		runTaskManager(tuple2._1(), resourceID, tuple2._2(), configuration, taskManagerClass);
    +	}
    +
    +	private static Tuple2<String, Integer> selectNetworkInterfaceAndPort(Configuration configuration)
    +		throws Exception {
    +		String taskManagerHostname = configuration.getString(ConfigConstants.TASK_MANAGER_HOSTNAME_KEY, null);
    +		if (taskManagerHostname != null) {
    +			LOG.info("Using configured hostname/address for TaskManager: " + taskManagerHostname);
    +		} else {
    +			LeaderRetrievalService leaderRetrievalService = LeaderRetrievalUtils.createLeaderRetrievalService(configuration);
    +			FiniteDuration lookupTimeout = AkkaUtils.getLookupTimeout(configuration);
    +
    +			InetAddress taskManagerAddress = LeaderRetrievalUtils.findConnectingAddress(leaderRetrievalService, lookupTimeout);
    +			taskManagerHostname = taskManagerAddress.getHostName();
    +			LOG.info("TaskManager will use hostname/address '{}' ({}) for communication.",
    +				taskManagerHostname, taskManagerAddress.getHostAddress());
    +		}
    +
    +		// if no task manager port has been configured, use 0 (system will pick any free port)
    +		int actorSystemPort = configuration.getInteger(ConfigConstants.TASK_MANAGER_IPC_PORT_KEY, 0);
    +		if (actorSystemPort < 0 || actorSystemPort > 65535) {
    +			throw new IllegalConfigurationException("Invalid value for '" +
    +				ConfigConstants.TASK_MANAGER_IPC_PORT_KEY +
    +				"' (port for the TaskManager actor system) : " + actorSystemPort +
    +				" - Leave config parameter empty or use 0 to let the system choose a port automatically.");
    +		}
    +
    +		return new Tuple2<>(taskManagerHostname, actorSystemPort);
    +	}
    +
    +	/**
    +	 * Starts and runs the TaskManager. Brings up an actor system for the TaskManager and its
    +	 * actors, starts the TaskManager's services (library cache, shuffle network stack, ...),
    +	 * and starts the TaskManager itself.
    +	 * <p/>
    +	 * This method will also spawn a process reaper for the TaskManager (kill the process if
    +	 * the actor fails) and optionally start the JVM memory logging thread.
    +	 *
    +	 * @param taskManagerHostname The hostname/address of the interface where the actor system
    +	 *                            will communicate.
    +	 * @param resourceID          The id of the resource which the task manager will run on.
    +	 * @param actorSystemPort     The port at which the actor system will communicate.
    +	 * @param configuration       The configuration for the TaskManager.
    +	 * @param taskManagerClass    The actor class to instantiate. Allows the use of TaskManager
    +	 *                            subclasses for example for YARN.
    +	 */
    +	private static void runTaskManager(
    +		String taskManagerHostname,
    +		ResourceID resourceID,
    +		int actorSystemPort,
    +		Configuration configuration,
    +		Class<? extends TaskManager> taskManagerClass) throws Exception {
    +
    +		LOG.info("Starting TaskManager");
    +
    +		// Bring up the TaskManager actor system first, bind it to the given address.
    +
    +		LOG.info("Starting TaskManager actor system at " +
    +			NetUtils.hostAndPortToUrlString(taskManagerHostname, actorSystemPort));
    +
    +		ActorSystem taskManagerSystem;
    +		try {
    +			Tuple2<String, Object> address = new Tuple2<String, Object>(taskManagerHostname, actorSystemPort);
    +			Config akkaConfig = AkkaUtils.getAkkaConfig(configuration, new Some<>(address));
    +			LOG.debug("Using akka configuration\n " + akkaConfig);
    +			taskManagerSystem = AkkaUtils.createActorSystem(akkaConfig);
    +		} catch (Throwable t) {
    +			if (t instanceof org.jboss.netty.channel.ChannelException) {
    +				Throwable cause = t.getCause();
    +				if (cause != null && t.getCause() instanceof java.net.BindException) {
    +					String address = NetUtils.hostAndPortToUrlString(taskManagerHostname, actorSystemPort);
    +					throw new IOException("Unable to bind TaskManager actor system to address " +
    +						address + " - " + cause.getMessage(), t);
    +				}
    +			}
    +			throw new Exception("Could not create TaskManager actor system", t);
    +		}
    +
    +		// start all the TaskManager services (network stack,  library cache, ...)
    +		// and the TaskManager actor
    +		try {
    +			LOG.info("Starting TaskManager actor");
    +			ActorRef taskManagerActor = startTaskManagerComponentsAndActor(
    +				configuration,
    +				resourceID,
    +				taskManagerSystem,
    +				taskManagerHostname,
    +				TASK_MANAGER_NAME,
    +				null,
    +				false,
    +				taskManagerClass);
    +
    +			// start a process reaper that watches the JobManager. If the TaskManager actor dies,
    +			// the process reaper will kill the JVM process (to ensure easy failure detection)
    +			LOG.debug("Starting TaskManager process reaper");
    +
    +			taskManagerSystem.actorOf(
    +				Props.create(ProcessReaper.class, taskManagerActor, LOG, RUNTIME_FAILURE_RETURN_CODE),
    +				"TaskManager_Process_Reaper");
    +
    +			// if desired, start the logging daemon that periodically logs the memory usage information
    +			if (LOG.isInfoEnabled() && configuration.getBoolean(
    +				ConfigConstants.TASK_MANAGER_DEBUG_MEMORY_USAGE_START_LOG_THREAD,
    +				ConfigConstants.DEFAULT_TASK_MANAGER_DEBUG_MEMORY_USAGE_START_LOG_THREAD)) {
    +				LOG.info("Starting periodic memory usage logger");
    +
    +				long interval = configuration.getLong(
    +					ConfigConstants.TASK_MANAGER_DEBUG_MEMORY_USAGE_LOG_INTERVAL_MS,
    +					ConfigConstants.DEFAULT_TASK_MANAGER_DEBUG_MEMORY_USAGE_LOG_INTERVAL_MS);
    +
    +				MemoryLogger logger = new MemoryLogger(LOG, interval, taskManagerSystem);
    +				logger.start();
    +			}
    +
    +			// block until everything is done
    +			taskManagerSystem.awaitTermination();
    +		} catch (Throwable t) {
    +			LOG.error("Error while starting up taskManager", t);
    +			try {
    +				taskManagerSystem.shutdown();
    +			} catch (Throwable tt) {
    +				LOG.warn("Could not cleanly shut down actor system", tt);
    +			}
    +			throw t;
    +		}
    +	}
    +
    +	// --------------------------------------------------------------------------
    +	//  Starting and running the TaskManager
    +	// --------------------------------------------------------------------------
    +
    +	/**
    +	 * @param configuration                 The configuration for the TaskManager.
    +	 * @param resourceID                    The id of the resource which the task manager will run on.
    +	 * @param actorSystem                  The actor system that should run the TaskManager actor.
    +	 * @param taskManagerHostname       The hostname/address that describes the TaskManager's data location.
    +	 * @param taskManagerActorName      Optionally the name of the TaskManager actor. If none is given,
    +	 *                                      the actor will use a random name.
    +	 * @param leaderRetrievalService        Optionally, a leader retrieval service can be provided. If none is given,
    +	 *                                      then a LeaderRetrievalService is constructed from the configuration.
    +	 * @param localTaskManagerCommunication If true, the TaskManager will not initiate the TCP network stack.
    +	 * @param taskManagerClass      The class of the TaskManager actor. May be used to give
    +	 *                                      subclasses that understand additional actor messages.
    +	 * @return An ActorRef to the TaskManager actor.
    +	 * @throws org.apache.flink.configuration.IllegalConfigurationException     Thrown, if the given config contains illegal values.
    +	 * @throws java.io.IOException      Thrown, if any of the I/O components (such as buffer pools,
    +	 *                                       I/O manager, ...) cannot be properly started.
    +	 * @throws java.lang.Exception      Thrown is some other error occurs while parsing the configuration
    +	 *                                      or starting the TaskManager components.
    +	 */
    +	public static ActorRef startTaskManagerComponentsAndActor(
    +		Configuration configuration,
    +		ResourceID resourceID,
    +		ActorSystem actorSystem,
    +		String taskManagerHostname,
    +		String taskManagerActorName,
    +		LeaderRetrievalService leaderRetrievalService,
    +		boolean localTaskManagerCommunication,
    +		Class<? extends TaskManager> taskManagerClass) throws Exception {
    +
    +		Tuple4<TaskManagerConfiguration, NetworkEnvironmentConfiguration, InstanceConnectionInfo, MemoryType> tuple4
    +			= parseTaskManagerConfiguration(configuration, taskManagerHostname, localTaskManagerCommunication);
    +
    +		TaskManagerConfiguration taskManagerConfig = tuple4._1();
    +		NetworkEnvironmentConfiguration netConfig = tuple4._2();
    +		InstanceConnectionInfo connectionInfo = tuple4._3();
    +		MemoryType memType = tuple4._4();
    +
    +		// pre-start checks
    +		checkTempDirs(taskManagerConfig.tmpDirPaths());
    +
    +		ExecutionContext executionContext = ExecutionContexts$.MODULE$.fromExecutor(new ForkJoinPool());
    +
    +		// we start the network first, to make sure it can allocate its buffers first
    +		NetworkEnvironment network = new NetworkEnvironment(executionContext, taskManagerConfig.timeout(), netConfig);
    +
    +		// computing the amount of memory to use depends on how much memory is available
    +		// it strictly needs to happen AFTER the network stack has been initialized
    +
    +		// check if a value has been configured
    +		long configuredMemory = configuration.getLong(ConfigConstants.TASK_MANAGER_MEMORY_SIZE_KEY, -1L);
    +		checkConfigParameter(configuredMemory == -1 || configuredMemory > 0, configuredMemory,
    +			ConfigConstants.TASK_MANAGER_MEMORY_SIZE_KEY,
    +			"MemoryManager needs at least one MB of memory. " +
    +				"If you leave this config parameter empty, the system automatically " +
    +				"pick a fraction of the available memory.");
    +
    +		long memorySize;
    +		boolean preAllocateMemory = configuration.getBoolean(
    +			ConfigConstants.TASK_MANAGER_MEMORY_PRE_ALLOCATE_KEY,
    +			ConfigConstants.DEFAULT_TASK_MANAGER_MEMORY_PRE_ALLOCATE);
    +		if (configuredMemory > 0) {
    +			if (preAllocateMemory) {
    +				LOG.info("Using {} MB for managed memory." , configuredMemory);
    +			} else {
    +				LOG.info("Limiting managed memory to {} MB, memory will be allocated lazily." , configuredMemory);
    +			}
    +			memorySize = configuredMemory << 20; // megabytes to bytes
    +		} else {
    +			float fraction = configuration.getFloat(
    +				ConfigConstants.TASK_MANAGER_MEMORY_FRACTION_KEY,
    +				ConfigConstants.DEFAULT_MEMORY_MANAGER_MEMORY_FRACTION);
    +			checkConfigParameter(fraction > 0.0f && fraction < 1.0f, fraction,
    +				ConfigConstants.TASK_MANAGER_MEMORY_FRACTION_KEY,
    +				"MemoryManager fraction of the free memory must be between 0.0 and 1.0");
    +
    +			if (memType == MemoryType.HEAP) {
    +				long relativeMemSize = (long) (EnvironmentInformation.getSizeOfFreeHeapMemoryWithDefrag() * fraction);
    +				if (preAllocateMemory) {
    +					LOG.info("Using {} of the currently free heap space for managed heap memory ({} MB)." ,
    +						fraction , relativeMemSize >> 20);
    +				} else {
    +					LOG.info("Limiting managed memory to {} of the currently free heap space ({} MB), " +
    +						"memory will be allocated lazily." , fraction , relativeMemSize >> 20);
    +				}
    +				memorySize = relativeMemSize;
    +			} else if (memType == MemoryType.OFF_HEAP) {
    +				// The maximum heap memory has been adjusted according to the fraction
    +				long maxMemory = EnvironmentInformation.getMaxJvmHeapMemory();
    +				long directMemorySize = (long) (maxMemory / (1.0 - fraction) * fraction);
    +				if (preAllocateMemory) {
    +					LOG.info("Using {} of the maximum memory size for managed off-heap memory ({} MB)." ,
    +						fraction, directMemorySize >> 20);
    +				} else {
    +					LOG.info("Limiting managed memory to {} of the maximum memory size ({} MB)," +
    +						" memory will be allocated lazily.", fraction, directMemorySize >> 20);
    +				}
    +				memorySize = directMemorySize;
    +			} else {
    +				throw new RuntimeException("No supported memory type detected.");
    +			}
    +		}
    +
    +		// now start the memory manager
    +		MemoryManager memoryManager;
    +		try {
    +			memoryManager = new MemoryManager(
    +				memorySize,
    +				taskManagerConfig.numberOfSlots(),
    +				netConfig.networkBufferSize(),
    +				memType,
    +				preAllocateMemory);
    +		} catch (OutOfMemoryError e) {
    +			if (memType == MemoryType.HEAP) {
    +				throw new Exception("OutOfMemory error (" + e.getMessage() +
    +					") while allocating the TaskManager heap memory (" + memorySize + " bytes).", e);
    +			} else if (memType == MemoryType.OFF_HEAP) {
    +				throw new Exception("OutOfMemory error (" + e.getMessage() +
    +					") while allocating the TaskManager off-heap memory (" + memorySize +
    +					" bytes).Try increasing the maximum direct memory (-XX:MaxDirectMemorySize)", e);
    +			} else {
    +				throw e;
    +			}
    +		}
    +
    +		// start the I/O manager, it will create some temp directories.
    +		IOManager ioManager = new IOManagerAsync(taskManagerConfig.tmpDirPaths());
    +
    +		if (leaderRetrievalService == null){
    +			leaderRetrievalService = LeaderRetrievalUtils.createLeaderRetrievalService(configuration);
    +		}
    +
    +		// create the actor properties (which define the actor constructor parameters)
    +		Props tmProps = Props.create(
    +			taskManagerClass,
    +			taskManagerConfig,
    +			resourceID,
    +			connectionInfo,
    +			memoryManager,
    +			ioManager,
    +			network,
    +			taskManagerConfig.numberOfSlots(),
    +			leaderRetrievalService);
    +
    +		ActorRef taskManagerActorRef;
    +		if (taskManagerActorName != null && !taskManagerActorName.equals("")) {
    +			taskManagerActorRef = actorSystem.actorOf(tmProps, taskManagerActorName);
    +		} else {
    +			taskManagerActorRef = actorSystem.actorOf(tmProps);
    +		}
    +
    +		return taskManagerActorRef;
    --- End diff --
    
    Yes, I agree with that. At the beginning I tried to construct the RpcService and HighAvailabilityService for new TaskExecutor. And the RpcService can be constructed based on ActorSystem, but how to construct HighAvailabilityService based on LeaderRetrievalService is not very clear currently, so I gave up this way for the first step. This work can be done when the HighAvailabilityService component to be more clearly later.


---
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.
---

[GitHub] flink pull request #2400: [FLINK-4363] Implement TaskManager basic startup o...

Posted by mxm <gi...@git.apache.org>.
Github user mxm commented on a diff in the pull request:

    https://github.com/apache/flink/pull/2400#discussion_r76085101
  
    --- Diff: flink-runtime/src/main/java/org/apache/flink/runtime/rpc/taskexecutor/TaskExecutor.java ---
    @@ -94,12 +729,11 @@ public void notifyOfNewResourceManagerLeader(String newLeaderAddress, UUID newLe
     			if (newLeaderAddress != null) {
     				// the resource manager switched to a new leader
     				log.info("ResourceManager leader changed from {} to {}. Registering at new leader.",
    -						resourceManagerConnection.getResourceManagerAddress(), newLeaderAddress);
    --- End diff --
    
    Why are these essentials RPC methods at the bottom. They should be moved to the top of the class.


---
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.
---

[GitHub] flink issue #2400: [FLINK-4363] Implement TaskManager basic startup of all c...

Posted by wangzhijiang999 <gi...@git.apache.org>.
Github user wangzhijiang999 commented on the issue:

    https://github.com/apache/flink/pull/2400
  
    Hi @mxm , I pushed some modifications based on your comments, including:
    1. Make var as final
    2. Implement TaskExecutorConfiguration class in java instead of TaskManagerConfiguration in scala, the NetworkEnvironmentConfiguration should also be rewrote by java, but it will cause import mistake in NetworkEnvironment, so retain the current implementation.
    3. Replace scala tuple2<hostname, port> by InetAddress, and the scala tuple4 by org.apache.flink.api.java.tuple.Tuple4
    4. Remove throws in checkConfigParameter method
    5. There are two public static methods  "selectNetworkInterfaceAndRunTaskManager" and "startTaskManagerComponentsAndActor" that can be invoked by outside world. For startTaskManagerComponentsAndActor method, the parameters should pass "RPCService" and  "HighAvailabilityServices" for constructing "TaskExecutor" directly. For "startTaskManagerComponentsAndActor" method, we will generate the default "RPCService" and "HighAvailabilityServices" or based on configuration. 
    6. Fix the missing ResourceID parameter description.
    
    Thank you for further suggestions!


---
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.
---

[GitHub] flink pull request #2400: [FLINK-4363] Implement TaskManager basic startup o...

Posted by wangzhijiang999 <gi...@git.apache.org>.
Github user wangzhijiang999 commented on a diff in the pull request:

    https://github.com/apache/flink/pull/2400#discussion_r75652404
  
    --- Diff: flink-runtime/src/main/java/org/apache/flink/runtime/rpc/taskexecutor/TaskExecutor.java ---
    @@ -35,27 +79,634 @@
      */
     public class TaskExecutor extends RpcEndpoint<TaskExecutorGateway> {
     
    -	/** The unique resource ID of this TaskExecutor */
    +	private static final Logger LOG = LoggerFactory.getLogger(TaskExecutor.class);
    +
    +	/** Return code for critical errors during the runtime */
    +	private static final int RUNTIME_FAILURE_RETURN_CODE = 2;
    +
    +	/** The name of the TaskManager actor */
    +	private static final String TASK_MANAGER_NAME = "taskmanager";
    --- End diff --
    
    Yes, the initial reason for this is that all the inner names still called "TaskManager", so the TaskExecutor class name can easily be be renamed to "TaskManager" to replace it in final. Otherwise we should replace all the "taskexecutor" into "taskmanager" after final replace.


---
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.
---

[GitHub] flink issue #2400: [FLINK-4363] Implement TaskManager basic startup of all c...

Posted by mxm <gi...@git.apache.org>.
Github user mxm commented on the issue:

    https://github.com/apache/flink/pull/2400
  
    Auto-closing from a commit doesn't in the feature branch. Could you please close the PR?


---
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.
---

[GitHub] flink pull request #2400: [FLINK-4363] Implement TaskManager basic startup o...

Posted by mxm <gi...@git.apache.org>.
Github user mxm commented on a diff in the pull request:

    https://github.com/apache/flink/pull/2400#discussion_r75646080
  
    --- Diff: flink-runtime/src/main/java/org/apache/flink/runtime/rpc/taskexecutor/TaskExecutor.java ---
    @@ -35,27 +79,634 @@
      */
     public class TaskExecutor extends RpcEndpoint<TaskExecutorGateway> {
     
    -	/** The unique resource ID of this TaskExecutor */
    +	private static final Logger LOG = LoggerFactory.getLogger(TaskExecutor.class);
    +
    +	/** Return code for critical errors during the runtime */
    +	private static final int RUNTIME_FAILURE_RETURN_CODE = 2;
    +
    +	/** The name of the TaskManager actor */
    +	private static final String TASK_MANAGER_NAME = "taskmanager";
    +
    +	/** The unique resource ID of this TaskManager */
     	private final ResourceID resourceID;
     
     	/** The access to the leader election and metadata storage services */
     	private final HighAvailabilityServices haServices;
     
    -	// --------- resource manager --------
    +	/** The task manager configuration */
    +	private final TaskManagerConfiguration taskManagerConfig;
    +
    +	/** The connection information of the task manager */
    +	private final InstanceConnectionInfo connectionInfo;
    +
    +	/** The I/O manager component in the task manager */
    +	private final IOManager ioManager;
    +
    +	/** The memory manager component in the task manager */
    +	private final MemoryManager memoryManager;
    +
    +	/** The network component in the task manager */
    +	private final NetworkEnvironment networkEnvironment;
    +
    +	/** The number of slots in the task manager, should be 1 for YARN */
    +	private final int numberOfSlots;
     
    +	// --------- resource manager --------
     	private TaskExecutorToResourceManagerConnection resourceManagerConnection;
     
     	// ------------------------------------------------------------------------
     
     	public TaskExecutor(
    +			TaskManagerConfiguration taskManagerConfig,
    +			ResourceID resourceID,
    +			InstanceConnectionInfo connectionInfo,
    +			MemoryManager memoryManager,
    +			IOManager ioManager,
    +			NetworkEnvironment networkEnvironment,
    +			int numberOfSlots,
     			RpcService rpcService,
    -			HighAvailabilityServices haServices,
    -			ResourceID resourceID) {
    +			HighAvailabilityServices haServices) {
     
     		super(rpcService);
     
    -		this.haServices = checkNotNull(haServices);
    +		this.taskManagerConfig = checkNotNull(taskManagerConfig);
     		this.resourceID = checkNotNull(resourceID);
    +		this.connectionInfo = checkNotNull(connectionInfo);
    +		this.memoryManager = checkNotNull(memoryManager);
    +		this.ioManager = checkNotNull(ioManager);
    +		this.networkEnvironment = checkNotNull(networkEnvironment);
    +		this.numberOfSlots = checkNotNull(numberOfSlots);
    +		this.haServices = checkNotNull(haServices);
    +	}
    +
    +	/**
    +	 * Starts and runs the TaskManager.
    +	 * <p/>
    +	 * This method first tries to select the network interface to use for the TaskManager
    +	 * communication. The network interface is used both for the actor communication
    +	 * (coordination) as well as for the data exchange between task managers. Unless
    +	 * the hostname/interface is explicitly configured in the configuration, this
    +	 * method will try out various interfaces and methods to connect to the JobManager
    +	 * and select the one where the connection attempt is successful.
    +	 * <p/>
    +	 * After selecting the network interface, this method brings up an actor system
    +	 * for the TaskManager and its actors, starts the TaskManager's services
    +	 * (library cache, shuffle network stack, ...), and starts the TaskManager itself.
    +	 *
    +	 * @param configuration    The configuration for the TaskManager.
    +	 * @param taskManagerClass The actor class to instantiate.
    +	 *                         Allows to use TaskManager subclasses for example for YARN.
    +	 */
    +	public static void selectNetworkInterfaceAndRunTaskManager(
    +		Configuration configuration,
    +		ResourceID resourceID,
    +		Class<? extends TaskManager> taskManagerClass) throws Exception {
    +
    +		Tuple2<String, Integer> tuple2 = selectNetworkInterfaceAndPort(configuration);
    +
    +		runTaskManager(tuple2._1(), resourceID, tuple2._2(), configuration, taskManagerClass);
    +	}
    +
    +	private static Tuple2<String, Integer> selectNetworkInterfaceAndPort(Configuration configuration)
    --- End diff --
    
    Do we want to rely on Scala tuples here after the conversion from Scala to Java? I would suggest `InetAddres` as a return type here.



---
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.
---

[GitHub] flink pull request #2400: [FLINK-4363] Implement TaskManager basic startup o...

Posted by mxm <gi...@git.apache.org>.
Github user mxm commented on a diff in the pull request:

    https://github.com/apache/flink/pull/2400#discussion_r75648199
  
    --- Diff: flink-runtime/src/main/java/org/apache/flink/runtime/rpc/taskexecutor/TaskExecutor.java ---
    @@ -35,27 +79,634 @@
      */
     public class TaskExecutor extends RpcEndpoint<TaskExecutorGateway> {
     
    -	/** The unique resource ID of this TaskExecutor */
    +	private static final Logger LOG = LoggerFactory.getLogger(TaskExecutor.class);
    +
    +	/** Return code for critical errors during the runtime */
    +	private static final int RUNTIME_FAILURE_RETURN_CODE = 2;
    +
    +	/** The name of the TaskManager actor */
    +	private static final String TASK_MANAGER_NAME = "taskmanager";
    +
    +	/** The unique resource ID of this TaskManager */
     	private final ResourceID resourceID;
     
     	/** The access to the leader election and metadata storage services */
     	private final HighAvailabilityServices haServices;
     
    -	// --------- resource manager --------
    +	/** The task manager configuration */
    +	private final TaskManagerConfiguration taskManagerConfig;
    +
    +	/** The connection information of the task manager */
    +	private final InstanceConnectionInfo connectionInfo;
    +
    +	/** The I/O manager component in the task manager */
    +	private final IOManager ioManager;
    +
    +	/** The memory manager component in the task manager */
    +	private final MemoryManager memoryManager;
    +
    +	/** The network component in the task manager */
    +	private final NetworkEnvironment networkEnvironment;
    +
    +	/** The number of slots in the task manager, should be 1 for YARN */
    +	private final int numberOfSlots;
     
    +	// --------- resource manager --------
     	private TaskExecutorToResourceManagerConnection resourceManagerConnection;
     
     	// ------------------------------------------------------------------------
     
     	public TaskExecutor(
    +			TaskManagerConfiguration taskManagerConfig,
    +			ResourceID resourceID,
    +			InstanceConnectionInfo connectionInfo,
    +			MemoryManager memoryManager,
    +			IOManager ioManager,
    +			NetworkEnvironment networkEnvironment,
    +			int numberOfSlots,
     			RpcService rpcService,
    -			HighAvailabilityServices haServices,
    -			ResourceID resourceID) {
    +			HighAvailabilityServices haServices) {
     
     		super(rpcService);
     
    -		this.haServices = checkNotNull(haServices);
    +		this.taskManagerConfig = checkNotNull(taskManagerConfig);
     		this.resourceID = checkNotNull(resourceID);
    +		this.connectionInfo = checkNotNull(connectionInfo);
    +		this.memoryManager = checkNotNull(memoryManager);
    +		this.ioManager = checkNotNull(ioManager);
    +		this.networkEnvironment = checkNotNull(networkEnvironment);
    +		this.numberOfSlots = checkNotNull(numberOfSlots);
    +		this.haServices = checkNotNull(haServices);
    +	}
    +
    +	/**
    +	 * Starts and runs the TaskManager.
    +	 * <p/>
    +	 * This method first tries to select the network interface to use for the TaskManager
    +	 * communication. The network interface is used both for the actor communication
    +	 * (coordination) as well as for the data exchange between task managers. Unless
    +	 * the hostname/interface is explicitly configured in the configuration, this
    +	 * method will try out various interfaces and methods to connect to the JobManager
    +	 * and select the one where the connection attempt is successful.
    +	 * <p/>
    +	 * After selecting the network interface, this method brings up an actor system
    +	 * for the TaskManager and its actors, starts the TaskManager's services
    +	 * (library cache, shuffle network stack, ...), and starts the TaskManager itself.
    +	 *
    +	 * @param configuration    The configuration for the TaskManager.
    +	 * @param taskManagerClass The actor class to instantiate.
    +	 *                         Allows to use TaskManager subclasses for example for YARN.
    +	 */
    +	public static void selectNetworkInterfaceAndRunTaskManager(
    +		Configuration configuration,
    +		ResourceID resourceID,
    +		Class<? extends TaskManager> taskManagerClass) throws Exception {
    +
    +		Tuple2<String, Integer> tuple2 = selectNetworkInterfaceAndPort(configuration);
    +
    +		runTaskManager(tuple2._1(), resourceID, tuple2._2(), configuration, taskManagerClass);
    +	}
    +
    +	private static Tuple2<String, Integer> selectNetworkInterfaceAndPort(Configuration configuration)
    +		throws Exception {
    +		String taskManagerHostname = configuration.getString(ConfigConstants.TASK_MANAGER_HOSTNAME_KEY, null);
    +		if (taskManagerHostname != null) {
    +			LOG.info("Using configured hostname/address for TaskManager: " + taskManagerHostname);
    +		} else {
    +			LeaderRetrievalService leaderRetrievalService = LeaderRetrievalUtils.createLeaderRetrievalService(configuration);
    +			FiniteDuration lookupTimeout = AkkaUtils.getLookupTimeout(configuration);
    +
    +			InetAddress taskManagerAddress = LeaderRetrievalUtils.findConnectingAddress(leaderRetrievalService, lookupTimeout);
    +			taskManagerHostname = taskManagerAddress.getHostName();
    +			LOG.info("TaskManager will use hostname/address '{}' ({}) for communication.",
    +				taskManagerHostname, taskManagerAddress.getHostAddress());
    +		}
    +
    +		// if no task manager port has been configured, use 0 (system will pick any free port)
    +		int actorSystemPort = configuration.getInteger(ConfigConstants.TASK_MANAGER_IPC_PORT_KEY, 0);
    +		if (actorSystemPort < 0 || actorSystemPort > 65535) {
    +			throw new IllegalConfigurationException("Invalid value for '" +
    +				ConfigConstants.TASK_MANAGER_IPC_PORT_KEY +
    +				"' (port for the TaskManager actor system) : " + actorSystemPort +
    +				" - Leave config parameter empty or use 0 to let the system choose a port automatically.");
    +		}
    +
    +		return new Tuple2<>(taskManagerHostname, actorSystemPort);
    +	}
    +
    +	/**
    +	 * Starts and runs the TaskManager. Brings up an actor system for the TaskManager and its
    +	 * actors, starts the TaskManager's services (library cache, shuffle network stack, ...),
    +	 * and starts the TaskManager itself.
    +	 * <p/>
    +	 * This method will also spawn a process reaper for the TaskManager (kill the process if
    +	 * the actor fails) and optionally start the JVM memory logging thread.
    +	 *
    +	 * @param taskManagerHostname The hostname/address of the interface where the actor system
    +	 *                            will communicate.
    +	 * @param resourceID          The id of the resource which the task manager will run on.
    +	 * @param actorSystemPort     The port at which the actor system will communicate.
    +	 * @param configuration       The configuration for the TaskManager.
    +	 * @param taskManagerClass    The actor class to instantiate. Allows the use of TaskManager
    +	 *                            subclasses for example for YARN.
    +	 */
    +	private static void runTaskManager(
    +		String taskManagerHostname,
    +		ResourceID resourceID,
    +		int actorSystemPort,
    +		Configuration configuration,
    +		Class<? extends TaskManager> taskManagerClass) throws Exception {
    +
    +		LOG.info("Starting TaskManager");
    +
    +		// Bring up the TaskManager actor system first, bind it to the given address.
    +
    +		LOG.info("Starting TaskManager actor system at " +
    +			NetUtils.hostAndPortToUrlString(taskManagerHostname, actorSystemPort));
    +
    +		ActorSystem taskManagerSystem;
    +		try {
    +			Tuple2<String, Object> address = new Tuple2<String, Object>(taskManagerHostname, actorSystemPort);
    +			Config akkaConfig = AkkaUtils.getAkkaConfig(configuration, new Some<>(address));
    +			LOG.debug("Using akka configuration\n " + akkaConfig);
    +			taskManagerSystem = AkkaUtils.createActorSystem(akkaConfig);
    +		} catch (Throwable t) {
    +			if (t instanceof org.jboss.netty.channel.ChannelException) {
    +				Throwable cause = t.getCause();
    +				if (cause != null && t.getCause() instanceof java.net.BindException) {
    +					String address = NetUtils.hostAndPortToUrlString(taskManagerHostname, actorSystemPort);
    +					throw new IOException("Unable to bind TaskManager actor system to address " +
    +						address + " - " + cause.getMessage(), t);
    +				}
    +			}
    +			throw new Exception("Could not create TaskManager actor system", t);
    +		}
    +
    +		// start all the TaskManager services (network stack,  library cache, ...)
    +		// and the TaskManager actor
    +		try {
    +			LOG.info("Starting TaskManager actor");
    +			ActorRef taskManagerActor = startTaskManagerComponentsAndActor(
    +				configuration,
    +				resourceID,
    +				taskManagerSystem,
    +				taskManagerHostname,
    +				TASK_MANAGER_NAME,
    +				null,
    +				false,
    +				taskManagerClass);
    +
    +			// start a process reaper that watches the JobManager. If the TaskManager actor dies,
    +			// the process reaper will kill the JVM process (to ensure easy failure detection)
    +			LOG.debug("Starting TaskManager process reaper");
    +
    +			taskManagerSystem.actorOf(
    +				Props.create(ProcessReaper.class, taskManagerActor, LOG, RUNTIME_FAILURE_RETURN_CODE),
    +				"TaskManager_Process_Reaper");
    +
    +			// if desired, start the logging daemon that periodically logs the memory usage information
    +			if (LOG.isInfoEnabled() && configuration.getBoolean(
    +				ConfigConstants.TASK_MANAGER_DEBUG_MEMORY_USAGE_START_LOG_THREAD,
    +				ConfigConstants.DEFAULT_TASK_MANAGER_DEBUG_MEMORY_USAGE_START_LOG_THREAD)) {
    +				LOG.info("Starting periodic memory usage logger");
    +
    +				long interval = configuration.getLong(
    +					ConfigConstants.TASK_MANAGER_DEBUG_MEMORY_USAGE_LOG_INTERVAL_MS,
    +					ConfigConstants.DEFAULT_TASK_MANAGER_DEBUG_MEMORY_USAGE_LOG_INTERVAL_MS);
    +
    +				MemoryLogger logger = new MemoryLogger(LOG, interval, taskManagerSystem);
    +				logger.start();
    +			}
    +
    +			// block until everything is done
    +			taskManagerSystem.awaitTermination();
    +		} catch (Throwable t) {
    +			LOG.error("Error while starting up taskManager", t);
    +			try {
    +				taskManagerSystem.shutdown();
    +			} catch (Throwable tt) {
    +				LOG.warn("Could not cleanly shut down actor system", tt);
    +			}
    +			throw t;
    +		}
    +	}
    +
    +	// --------------------------------------------------------------------------
    +	//  Starting and running the TaskManager
    +	// --------------------------------------------------------------------------
    +
    +	/**
    +	 * @param configuration                 The configuration for the TaskManager.
    +	 * @param resourceID                    The id of the resource which the task manager will run on.
    +	 * @param actorSystem                  The actor system that should run the TaskManager actor.
    +	 * @param taskManagerHostname       The hostname/address that describes the TaskManager's data location.
    +	 * @param taskManagerActorName      Optionally the name of the TaskManager actor. If none is given,
    +	 *                                      the actor will use a random name.
    +	 * @param leaderRetrievalService        Optionally, a leader retrieval service can be provided. If none is given,
    +	 *                                      then a LeaderRetrievalService is constructed from the configuration.
    +	 * @param localTaskManagerCommunication If true, the TaskManager will not initiate the TCP network stack.
    +	 * @param taskManagerClass      The class of the TaskManager actor. May be used to give
    +	 *                                      subclasses that understand additional actor messages.
    +	 * @return An ActorRef to the TaskManager actor.
    +	 * @throws org.apache.flink.configuration.IllegalConfigurationException     Thrown, if the given config contains illegal values.
    +	 * @throws java.io.IOException      Thrown, if any of the I/O components (such as buffer pools,
    +	 *                                       I/O manager, ...) cannot be properly started.
    +	 * @throws java.lang.Exception      Thrown is some other error occurs while parsing the configuration
    +	 *                                      or starting the TaskManager components.
    +	 */
    +	public static ActorRef startTaskManagerComponentsAndActor(
    +		Configuration configuration,
    +		ResourceID resourceID,
    +		ActorSystem actorSystem,
    +		String taskManagerHostname,
    +		String taskManagerActorName,
    +		LeaderRetrievalService leaderRetrievalService,
    +		boolean localTaskManagerCommunication,
    +		Class<? extends TaskManager> taskManagerClass) throws Exception {
    +
    +		Tuple4<TaskManagerConfiguration, NetworkEnvironmentConfiguration, InstanceConnectionInfo, MemoryType> tuple4
    +			= parseTaskManagerConfiguration(configuration, taskManagerHostname, localTaskManagerCommunication);
    +
    +		TaskManagerConfiguration taskManagerConfig = tuple4._1();
    +		NetworkEnvironmentConfiguration netConfig = tuple4._2();
    +		InstanceConnectionInfo connectionInfo = tuple4._3();
    +		MemoryType memType = tuple4._4();
    +
    +		// pre-start checks
    +		checkTempDirs(taskManagerConfig.tmpDirPaths());
    +
    +		ExecutionContext executionContext = ExecutionContexts$.MODULE$.fromExecutor(new ForkJoinPool());
    +
    +		// we start the network first, to make sure it can allocate its buffers first
    +		NetworkEnvironment network = new NetworkEnvironment(executionContext, taskManagerConfig.timeout(), netConfig);
    +
    +		// computing the amount of memory to use depends on how much memory is available
    +		// it strictly needs to happen AFTER the network stack has been initialized
    +
    +		// check if a value has been configured
    +		long configuredMemory = configuration.getLong(ConfigConstants.TASK_MANAGER_MEMORY_SIZE_KEY, -1L);
    +		checkConfigParameter(configuredMemory == -1 || configuredMemory > 0, configuredMemory,
    +			ConfigConstants.TASK_MANAGER_MEMORY_SIZE_KEY,
    +			"MemoryManager needs at least one MB of memory. " +
    +				"If you leave this config parameter empty, the system automatically " +
    +				"pick a fraction of the available memory.");
    +
    +		long memorySize;
    +		boolean preAllocateMemory = configuration.getBoolean(
    +			ConfigConstants.TASK_MANAGER_MEMORY_PRE_ALLOCATE_KEY,
    +			ConfigConstants.DEFAULT_TASK_MANAGER_MEMORY_PRE_ALLOCATE);
    +		if (configuredMemory > 0) {
    +			if (preAllocateMemory) {
    +				LOG.info("Using {} MB for managed memory." , configuredMemory);
    +			} else {
    +				LOG.info("Limiting managed memory to {} MB, memory will be allocated lazily." , configuredMemory);
    +			}
    +			memorySize = configuredMemory << 20; // megabytes to bytes
    +		} else {
    +			float fraction = configuration.getFloat(
    +				ConfigConstants.TASK_MANAGER_MEMORY_FRACTION_KEY,
    +				ConfigConstants.DEFAULT_MEMORY_MANAGER_MEMORY_FRACTION);
    +			checkConfigParameter(fraction > 0.0f && fraction < 1.0f, fraction,
    +				ConfigConstants.TASK_MANAGER_MEMORY_FRACTION_KEY,
    +				"MemoryManager fraction of the free memory must be between 0.0 and 1.0");
    +
    +			if (memType == MemoryType.HEAP) {
    +				long relativeMemSize = (long) (EnvironmentInformation.getSizeOfFreeHeapMemoryWithDefrag() * fraction);
    +				if (preAllocateMemory) {
    +					LOG.info("Using {} of the currently free heap space for managed heap memory ({} MB)." ,
    +						fraction , relativeMemSize >> 20);
    +				} else {
    +					LOG.info("Limiting managed memory to {} of the currently free heap space ({} MB), " +
    +						"memory will be allocated lazily." , fraction , relativeMemSize >> 20);
    +				}
    +				memorySize = relativeMemSize;
    +			} else if (memType == MemoryType.OFF_HEAP) {
    +				// The maximum heap memory has been adjusted according to the fraction
    +				long maxMemory = EnvironmentInformation.getMaxJvmHeapMemory();
    +				long directMemorySize = (long) (maxMemory / (1.0 - fraction) * fraction);
    +				if (preAllocateMemory) {
    +					LOG.info("Using {} of the maximum memory size for managed off-heap memory ({} MB)." ,
    +						fraction, directMemorySize >> 20);
    +				} else {
    +					LOG.info("Limiting managed memory to {} of the maximum memory size ({} MB)," +
    +						" memory will be allocated lazily.", fraction, directMemorySize >> 20);
    +				}
    +				memorySize = directMemorySize;
    +			} else {
    +				throw new RuntimeException("No supported memory type detected.");
    +			}
    +		}
    +
    +		// now start the memory manager
    +		MemoryManager memoryManager;
    +		try {
    +			memoryManager = new MemoryManager(
    +				memorySize,
    +				taskManagerConfig.numberOfSlots(),
    +				netConfig.networkBufferSize(),
    +				memType,
    +				preAllocateMemory);
    +		} catch (OutOfMemoryError e) {
    +			if (memType == MemoryType.HEAP) {
    +				throw new Exception("OutOfMemory error (" + e.getMessage() +
    +					") while allocating the TaskManager heap memory (" + memorySize + " bytes).", e);
    +			} else if (memType == MemoryType.OFF_HEAP) {
    +				throw new Exception("OutOfMemory error (" + e.getMessage() +
    +					") while allocating the TaskManager off-heap memory (" + memorySize +
    +					" bytes).Try increasing the maximum direct memory (-XX:MaxDirectMemorySize)", e);
    +			} else {
    +				throw e;
    +			}
    +		}
    +
    +		// start the I/O manager, it will create some temp directories.
    +		IOManager ioManager = new IOManagerAsync(taskManagerConfig.tmpDirPaths());
    +
    +		if (leaderRetrievalService == null){
    +			leaderRetrievalService = LeaderRetrievalUtils.createLeaderRetrievalService(configuration);
    +		}
    +
    +		// create the actor properties (which define the actor constructor parameters)
    +		Props tmProps = Props.create(
    +			taskManagerClass,
    +			taskManagerConfig,
    +			resourceID,
    +			connectionInfo,
    +			memoryManager,
    +			ioManager,
    +			network,
    +			taskManagerConfig.numberOfSlots(),
    +			leaderRetrievalService);
    +
    +		ActorRef taskManagerActorRef;
    +		if (taskManagerActorName != null && !taskManagerActorName.equals("")) {
    +			taskManagerActorRef = actorSystem.actorOf(tmProps, taskManagerActorName);
    +		} else {
    +			taskManagerActorRef = actorSystem.actorOf(tmProps);
    +		}
    +
    +		return taskManagerActorRef;
    +	}
    +
    +	// --------------------------------------------------------------------------
    +	//  Parsing and checking the TaskManager Configuration
    +	// --------------------------------------------------------------------------
    +
    +	/**
    +	 * Utility method to extract TaskManager config parameters from the configuration and to
    +	 * sanity check them.
    +	 *
    +	 * @param configuration                 The configuration.
    +	 * @param taskManagerHostname           The host name under which the TaskManager communicates.
    +	 * @param localTaskManagerCommunication True, to skip initializing the network stack.
    +	 *                                      Use only in cases where only one task manager runs.
    +	 * @return A tuple (TaskManagerConfiguration, network configuration,
    +	 * InstanceConnectionInfo, JobManager actor Akka URL).
    +	 */
    +	private static Tuple4<TaskManagerConfiguration, NetworkEnvironmentConfiguration, InstanceConnectionInfo, MemoryType>
    +		parseTaskManagerConfiguration(Configuration configuration, String taskManagerHostname, boolean localTaskManagerCommunication)
    +		throws Exception {
    +
    +		// ------- read values from the config and check them ---------
    +		//                      (a lot of them)
    +
    +		// ----> hosts / ports for communication and data exchange
    +
    +		int dataport = configuration.getInteger(ConfigConstants.TASK_MANAGER_DATA_PORT_KEY,
    +			ConfigConstants.DEFAULT_TASK_MANAGER_DATA_PORT);
    +		if (dataport == 0) {
    +			dataport = NetUtils.getAvailablePort();
    +		}
    +		checkConfigParameter(dataport > 0, dataport, ConfigConstants.TASK_MANAGER_DATA_PORT_KEY,
    +			"Leave config parameter empty or use 0 to let the system choose a port automatically.");
    +
    +		InetAddress taskManagerAddress = InetAddress.getByName(taskManagerHostname);
    +		InstanceConnectionInfo connectionInfo = new InstanceConnectionInfo(taskManagerAddress, dataport);
    +
    +		// ----> memory / network stack (shuffles/broadcasts), task slots, temp directories
    +
    +		// we need this because many configs have been written with a "-1" entry
    +		int slots = configuration.getInteger(ConfigConstants.TASK_MANAGER_NUM_TASK_SLOTS, 1);
    +		if (slots == -1) {
    +			slots = 1;
    +		}
    +		checkConfigParameter(slots >= 1, slots, ConfigConstants.TASK_MANAGER_NUM_TASK_SLOTS,
    +			"Number of task slots must be at least one.");
    +
    +		int numNetworkBuffers = configuration.getInteger(
    +			ConfigConstants.TASK_MANAGER_NETWORK_NUM_BUFFERS_KEY,
    +			ConfigConstants.DEFAULT_TASK_MANAGER_NETWORK_NUM_BUFFERS);
    +		checkConfigParameter(numNetworkBuffers > 0, numNetworkBuffers,
    +			ConfigConstants.TASK_MANAGER_NETWORK_NUM_BUFFERS_KEY, "");
    +
    +		int pageSize = configuration.getInteger(
    +			ConfigConstants.TASK_MANAGER_MEMORY_SEGMENT_SIZE_KEY,
    +			ConfigConstants.DEFAULT_TASK_MANAGER_MEMORY_SEGMENT_SIZE);
    +		// check page size of for minimum size
    +		checkConfigParameter(pageSize >= MemoryManager.MIN_PAGE_SIZE, pageSize,
    +			ConfigConstants.TASK_MANAGER_MEMORY_SEGMENT_SIZE_KEY,
    +			"Minimum memory segment size is " + MemoryManager.MIN_PAGE_SIZE);
    +		// check page size for power of two
    +		checkConfigParameter(MathUtils.isPowerOf2(pageSize), pageSize,
    +			ConfigConstants.TASK_MANAGER_MEMORY_SEGMENT_SIZE_KEY,
    +			"Memory segment size must be a power of 2.");
    +
    +		// check whether we use heap or off-heap memory
    +		MemoryType memType;
    +		if (configuration.getBoolean(ConfigConstants.TASK_MANAGER_MEMORY_OFF_HEAP_KEY, false)) {
    +			memType = MemoryType.OFF_HEAP;
    +		} else {
    +			memType = MemoryType.HEAP;
    +		}
    +
    +		// initialize the memory segment factory accordingly
    +		if (memType == MemoryType.HEAP) {
    +			if (!MemorySegmentFactory.isInitialized()) {
    +				MemorySegmentFactory.initializeFactory(HeapMemorySegment.FACTORY);
    +			} else if (MemorySegmentFactory.getFactory() != HeapMemorySegment.FACTORY) {
    +				throw new Exception("Memory type is set to heap memory, but memory segment " +
    +					"factory has been initialized for off-heap memory segments");
    +			}
    +		} else {
    +			if (!MemorySegmentFactory.isInitialized()) {
    +				MemorySegmentFactory.initializeFactory(HybridMemorySegment.FACTORY);
    +			} else if (MemorySegmentFactory.getFactory() != HybridMemorySegment.FACTORY) {
    +				throw new Exception("Memory type is set to off-heap memory, but memory segment " +
    +					"factory has been initialized for heap memory segments");
    +			}
    +		}
    +
    +		String[] tmpDirs = configuration.getString(
    +			ConfigConstants.TASK_MANAGER_TMP_DIR_KEY,
    +			ConfigConstants.DEFAULT_TASK_MANAGER_TMP_PATH).split(",|" + File.pathSeparator);
    +
    +		NettyConfig nettyConfig = null;
    +		if (!localTaskManagerCommunication) {
    +			nettyConfig = new NettyConfig(connectionInfo.address(), connectionInfo.dataPort(), pageSize, slots, configuration);
    +		}
    +
    +		// Default spill I/O mode for intermediate results
    +		String syncOrAsync = configuration.getString(
    +			ConfigConstants.TASK_MANAGER_NETWORK_DEFAULT_IO_MODE,
    +			ConfigConstants.DEFAULT_TASK_MANAGER_NETWORK_DEFAULT_IO_MODE);
    +
    +		IOMode ioMode;
    +		if (syncOrAsync == "async") {
    +			ioMode = IOManager.IOMode.ASYNC;
    +		} else {
    +			ioMode = IOManager.IOMode.SYNC;
    +		}
    +
    +		NetworkEnvironmentConfiguration networkConfig = new NetworkEnvironmentConfiguration(
    +			numNetworkBuffers,
    +			pageSize,
    +			memType,
    +			ioMode,
    +			nettyConfig == null ? Option.<NettyConfig>empty() : new Some<>(nettyConfig),
    +			new Tuple2<>(500, 3000));
    +
    +		// ----> timeouts, library caching, profiling
    +
    +		FiniteDuration timeout;
    +		try {
    +			timeout = AkkaUtils.getTimeout(configuration);
    +		} catch (Exception e) {
    +			throw new IllegalArgumentException(
    +				"Invalid format for '" + ConfigConstants.AKKA_ASK_TIMEOUT +
    +					"'.Use formats like '50 s' or '1 min' to specify the timeout.");
    +		}
    +		LOG.info("Messages between TaskManager and JobManager have a max timeout of " + timeout);
    +
    +		long cleanupInterval = configuration.getLong(
    +			ConfigConstants.LIBRARY_CACHE_MANAGER_CLEANUP_INTERVAL,
    +			ConfigConstants.DEFAULT_LIBRARY_CACHE_MANAGER_CLEANUP_INTERVAL) * 1000;
    +
    +		FiniteDuration finiteRegistrationDuration = null;
    +		try {
    +			Duration maxRegistrationDuration = Duration.create(configuration.getString(
    +				ConfigConstants.TASK_MANAGER_MAX_REGISTRATION_DURATION,
    +				ConfigConstants.DEFAULT_TASK_MANAGER_MAX_REGISTRATION_DURATION));
    +			if (maxRegistrationDuration.isFinite()) {
    +				finiteRegistrationDuration = new FiniteDuration(maxRegistrationDuration.toSeconds(), TimeUnit.SECONDS);
    +			}
    +		} catch (NumberFormatException e) {
    +			throw new IllegalArgumentException("Invalid format for parameter " +
    +				ConfigConstants.TASK_MANAGER_MAX_REGISTRATION_DURATION, e);
    +		}
    +
    +		FiniteDuration initialRegistrationPause;
    +		try {
    +			Duration pause = Duration.create(configuration.getString(
    +				ConfigConstants.TASK_MANAGER_INITIAL_REGISTRATION_PAUSE,
    +				ConfigConstants.DEFAULT_TASK_MANAGER_INITIAL_REGISTRATION_PAUSE));
    +			if (pause.isFinite()) {
    +				initialRegistrationPause = new FiniteDuration(pause.toSeconds(), TimeUnit.SECONDS);
    +			} else {
    +				throw new IllegalArgumentException("The initial registration pause must be finite: " + pause);
    +			}
    +		} catch (NumberFormatException e) {
    +			throw new IllegalArgumentException("Invalid format for parameter " +
    +				ConfigConstants.TASK_MANAGER_INITIAL_REGISTRATION_PAUSE, e);
    +		}
    +
    +		FiniteDuration maxRegistrationPause;
    +		try {
    +			Duration pause = Duration.create(configuration.getString(
    +				ConfigConstants.TASK_MANAGER_MAX_REGISTARTION_PAUSE,
    +				ConfigConstants.DEFAULT_TASK_MANAGER_MAX_REGISTRATION_PAUSE));
    +			if (pause.isFinite()) {
    +				maxRegistrationPause = new FiniteDuration(pause.toSeconds(), TimeUnit.SECONDS);
    +			} else {
    +				throw new IllegalArgumentException("The maximum registration pause must be finite: " + pause);
    +			}
    +		} catch (NumberFormatException e) {
    +			throw new IllegalArgumentException("Invalid format for parameter " +
    +				ConfigConstants.TASK_MANAGER_INITIAL_REGISTRATION_PAUSE, e);
    +		}
    +
    +		FiniteDuration refusedRegistrationPause;
    +		try {
    +			Duration pause = Duration.create(configuration.getString(
    +				ConfigConstants.TASK_MANAGER_REFUSED_REGISTRATION_PAUSE,
    +				ConfigConstants.DEFAULT_TASK_MANAGER_REFUSED_REGISTRATION_PAUSE));
    +			if (pause.isFinite()) {
    +				refusedRegistrationPause = new FiniteDuration(pause.toSeconds(), TimeUnit.SECONDS);
    +			} else {
    +				throw new IllegalArgumentException("The refused registration pause must be finite: " + pause);
    +			}
    +		} catch (NumberFormatException e) {
    +			throw new IllegalArgumentException("Invalid format for parameter " +
    +				ConfigConstants.TASK_MANAGER_INITIAL_REGISTRATION_PAUSE, e);
    +		}
    +
    +		TaskManagerConfiguration taskManagerConfig = new TaskManagerConfiguration(
    +			tmpDirs,
    +			cleanupInterval,
    +			timeout,
    +			finiteRegistrationDuration == null ? Option.<FiniteDuration>empty() : new Some<>(finiteRegistrationDuration),
    +			slots,
    +			configuration,
    +			initialRegistrationPause,
    +			maxRegistrationPause,
    +			refusedRegistrationPause);
    +
    +		return new Tuple4<>(taskManagerConfig, networkConfig, connectionInfo, memType);
    +	}
    +
    +	/**
    +	 * Validates a condition for a config parameter and displays a standard exception, if the
    +	 * the condition does not hold.
    +	 *
    +	 * @param condition    The condition that must hold. If the condition is false, an exception is thrown.
    +	 * @param parameter    The parameter value. Will be shown in the exception message.
    +	 * @param name         The name of the config parameter. Will be shown in the exception message.
    +	 * @param errorMessage The optional custom error message to append to the exception message.
    +	 * @throws IllegalConfigurationException Thrown if the condition is violated.
    +	 */
    +	private static void checkConfigParameter(
    +		boolean condition,
    +		Object parameter,
    +		String name,
    +		String errorMessage) throws IllegalConfigurationException {
    --- End diff --
    
    I think the throws is not necessary because it is a "RuntimeException".


---
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.
---

[GitHub] flink pull request #2400: [FLINK-4363] Implement TaskManager basic startup o...

Posted by mxm <gi...@git.apache.org>.
Github user mxm commented on a diff in the pull request:

    https://github.com/apache/flink/pull/2400#discussion_r75680482
  
    --- Diff: flink-runtime/src/main/java/org/apache/flink/runtime/rpc/taskexecutor/TaskExecutor.java ---
    @@ -35,27 +79,634 @@
      */
     public class TaskExecutor extends RpcEndpoint<TaskExecutorGateway> {
     
    -	/** The unique resource ID of this TaskExecutor */
    +	private static final Logger LOG = LoggerFactory.getLogger(TaskExecutor.class);
    +
    +	/** Return code for critical errors during the runtime */
    +	private static final int RUNTIME_FAILURE_RETURN_CODE = 2;
    +
    +	/** The name of the TaskManager actor */
    +	private static final String TASK_MANAGER_NAME = "taskmanager";
    +
    +	/** The unique resource ID of this TaskManager */
     	private final ResourceID resourceID;
     
     	/** The access to the leader election and metadata storage services */
     	private final HighAvailabilityServices haServices;
     
    -	// --------- resource manager --------
    +	/** The task manager configuration */
    +	private final TaskManagerConfiguration taskManagerConfig;
    +
    +	/** The connection information of the task manager */
    +	private final InstanceConnectionInfo connectionInfo;
    +
    +	/** The I/O manager component in the task manager */
    +	private final IOManager ioManager;
    +
    +	/** The memory manager component in the task manager */
    +	private final MemoryManager memoryManager;
    +
    +	/** The network component in the task manager */
    +	private final NetworkEnvironment networkEnvironment;
    +
    +	/** The number of slots in the task manager, should be 1 for YARN */
    +	private final int numberOfSlots;
     
    +	// --------- resource manager --------
     	private TaskExecutorToResourceManagerConnection resourceManagerConnection;
     
     	// ------------------------------------------------------------------------
     
     	public TaskExecutor(
    +			TaskManagerConfiguration taskManagerConfig,
    +			ResourceID resourceID,
    +			InstanceConnectionInfo connectionInfo,
    +			MemoryManager memoryManager,
    +			IOManager ioManager,
    +			NetworkEnvironment networkEnvironment,
    +			int numberOfSlots,
     			RpcService rpcService,
    -			HighAvailabilityServices haServices,
    -			ResourceID resourceID) {
    +			HighAvailabilityServices haServices) {
     
     		super(rpcService);
     
    -		this.haServices = checkNotNull(haServices);
    +		this.taskManagerConfig = checkNotNull(taskManagerConfig);
     		this.resourceID = checkNotNull(resourceID);
    +		this.connectionInfo = checkNotNull(connectionInfo);
    +		this.memoryManager = checkNotNull(memoryManager);
    +		this.ioManager = checkNotNull(ioManager);
    +		this.networkEnvironment = checkNotNull(networkEnvironment);
    +		this.numberOfSlots = checkNotNull(numberOfSlots);
    +		this.haServices = checkNotNull(haServices);
    +	}
    +
    +	/**
    +	 * Starts and runs the TaskManager.
    +	 * <p/>
    +	 * This method first tries to select the network interface to use for the TaskManager
    +	 * communication. The network interface is used both for the actor communication
    +	 * (coordination) as well as for the data exchange between task managers. Unless
    +	 * the hostname/interface is explicitly configured in the configuration, this
    +	 * method will try out various interfaces and methods to connect to the JobManager
    +	 * and select the one where the connection attempt is successful.
    +	 * <p/>
    +	 * After selecting the network interface, this method brings up an actor system
    +	 * for the TaskManager and its actors, starts the TaskManager's services
    +	 * (library cache, shuffle network stack, ...), and starts the TaskManager itself.
    +	 *
    +	 * @param configuration    The configuration for the TaskManager.
    +	 * @param taskManagerClass The actor class to instantiate.
    +	 *                         Allows to use TaskManager subclasses for example for YARN.
    +	 */
    +	public static void selectNetworkInterfaceAndRunTaskManager(
    +		Configuration configuration,
    +		ResourceID resourceID,
    +		Class<? extends TaskManager> taskManagerClass) throws Exception {
    +
    +		Tuple2<String, Integer> tuple2 = selectNetworkInterfaceAndPort(configuration);
    +
    +		runTaskManager(tuple2._1(), resourceID, tuple2._2(), configuration, taskManagerClass);
    +	}
    +
    +	private static Tuple2<String, Integer> selectNetworkInterfaceAndPort(Configuration configuration)
    +		throws Exception {
    +		String taskManagerHostname = configuration.getString(ConfigConstants.TASK_MANAGER_HOSTNAME_KEY, null);
    +		if (taskManagerHostname != null) {
    +			LOG.info("Using configured hostname/address for TaskManager: " + taskManagerHostname);
    +		} else {
    +			LeaderRetrievalService leaderRetrievalService = LeaderRetrievalUtils.createLeaderRetrievalService(configuration);
    +			FiniteDuration lookupTimeout = AkkaUtils.getLookupTimeout(configuration);
    +
    +			InetAddress taskManagerAddress = LeaderRetrievalUtils.findConnectingAddress(leaderRetrievalService, lookupTimeout);
    +			taskManagerHostname = taskManagerAddress.getHostName();
    +			LOG.info("TaskManager will use hostname/address '{}' ({}) for communication.",
    +				taskManagerHostname, taskManagerAddress.getHostAddress());
    +		}
    +
    +		// if no task manager port has been configured, use 0 (system will pick any free port)
    +		int actorSystemPort = configuration.getInteger(ConfigConstants.TASK_MANAGER_IPC_PORT_KEY, 0);
    +		if (actorSystemPort < 0 || actorSystemPort > 65535) {
    +			throw new IllegalConfigurationException("Invalid value for '" +
    +				ConfigConstants.TASK_MANAGER_IPC_PORT_KEY +
    +				"' (port for the TaskManager actor system) : " + actorSystemPort +
    +				" - Leave config parameter empty or use 0 to let the system choose a port automatically.");
    +		}
    +
    +		return new Tuple2<>(taskManagerHostname, actorSystemPort);
    +	}
    +
    +	/**
    +	 * Starts and runs the TaskManager. Brings up an actor system for the TaskManager and its
    +	 * actors, starts the TaskManager's services (library cache, shuffle network stack, ...),
    +	 * and starts the TaskManager itself.
    +	 * <p/>
    +	 * This method will also spawn a process reaper for the TaskManager (kill the process if
    +	 * the actor fails) and optionally start the JVM memory logging thread.
    +	 *
    +	 * @param taskManagerHostname The hostname/address of the interface where the actor system
    +	 *                            will communicate.
    +	 * @param resourceID          The id of the resource which the task manager will run on.
    +	 * @param actorSystemPort     The port at which the actor system will communicate.
    +	 * @param configuration       The configuration for the TaskManager.
    +	 * @param taskManagerClass    The actor class to instantiate. Allows the use of TaskManager
    +	 *                            subclasses for example for YARN.
    +	 */
    +	private static void runTaskManager(
    +		String taskManagerHostname,
    +		ResourceID resourceID,
    +		int actorSystemPort,
    +		Configuration configuration,
    +		Class<? extends TaskManager> taskManagerClass) throws Exception {
    +
    +		LOG.info("Starting TaskManager");
    +
    +		// Bring up the TaskManager actor system first, bind it to the given address.
    +
    +		LOG.info("Starting TaskManager actor system at " +
    +			NetUtils.hostAndPortToUrlString(taskManagerHostname, actorSystemPort));
    +
    +		ActorSystem taskManagerSystem;
    +		try {
    +			Tuple2<String, Object> address = new Tuple2<String, Object>(taskManagerHostname, actorSystemPort);
    +			Config akkaConfig = AkkaUtils.getAkkaConfig(configuration, new Some<>(address));
    +			LOG.debug("Using akka configuration\n " + akkaConfig);
    +			taskManagerSystem = AkkaUtils.createActorSystem(akkaConfig);
    +		} catch (Throwable t) {
    +			if (t instanceof org.jboss.netty.channel.ChannelException) {
    +				Throwable cause = t.getCause();
    +				if (cause != null && t.getCause() instanceof java.net.BindException) {
    +					String address = NetUtils.hostAndPortToUrlString(taskManagerHostname, actorSystemPort);
    +					throw new IOException("Unable to bind TaskManager actor system to address " +
    +						address + " - " + cause.getMessage(), t);
    +				}
    +			}
    +			throw new Exception("Could not create TaskManager actor system", t);
    +		}
    +
    +		// start all the TaskManager services (network stack,  library cache, ...)
    +		// and the TaskManager actor
    +		try {
    +			LOG.info("Starting TaskManager actor");
    +			ActorRef taskManagerActor = startTaskManagerComponentsAndActor(
    +				configuration,
    +				resourceID,
    +				taskManagerSystem,
    +				taskManagerHostname,
    +				TASK_MANAGER_NAME,
    +				null,
    +				false,
    +				taskManagerClass);
    +
    +			// start a process reaper that watches the JobManager. If the TaskManager actor dies,
    +			// the process reaper will kill the JVM process (to ensure easy failure detection)
    +			LOG.debug("Starting TaskManager process reaper");
    +
    +			taskManagerSystem.actorOf(
    +				Props.create(ProcessReaper.class, taskManagerActor, LOG, RUNTIME_FAILURE_RETURN_CODE),
    +				"TaskManager_Process_Reaper");
    +
    +			// if desired, start the logging daemon that periodically logs the memory usage information
    +			if (LOG.isInfoEnabled() && configuration.getBoolean(
    +				ConfigConstants.TASK_MANAGER_DEBUG_MEMORY_USAGE_START_LOG_THREAD,
    +				ConfigConstants.DEFAULT_TASK_MANAGER_DEBUG_MEMORY_USAGE_START_LOG_THREAD)) {
    +				LOG.info("Starting periodic memory usage logger");
    +
    +				long interval = configuration.getLong(
    +					ConfigConstants.TASK_MANAGER_DEBUG_MEMORY_USAGE_LOG_INTERVAL_MS,
    +					ConfigConstants.DEFAULT_TASK_MANAGER_DEBUG_MEMORY_USAGE_LOG_INTERVAL_MS);
    +
    +				MemoryLogger logger = new MemoryLogger(LOG, interval, taskManagerSystem);
    +				logger.start();
    +			}
    +
    +			// block until everything is done
    +			taskManagerSystem.awaitTermination();
    +		} catch (Throwable t) {
    +			LOG.error("Error while starting up taskManager", t);
    +			try {
    +				taskManagerSystem.shutdown();
    +			} catch (Throwable tt) {
    +				LOG.warn("Could not cleanly shut down actor system", tt);
    +			}
    +			throw t;
    +		}
    +	}
    +
    +	// --------------------------------------------------------------------------
    +	//  Starting and running the TaskManager
    +	// --------------------------------------------------------------------------
    +
    +	/**
    +	 * @param configuration                 The configuration for the TaskManager.
    +	 * @param resourceID                    The id of the resource which the task manager will run on.
    +	 * @param actorSystem                  The actor system that should run the TaskManager actor.
    +	 * @param taskManagerHostname       The hostname/address that describes the TaskManager's data location.
    +	 * @param taskManagerActorName      Optionally the name of the TaskManager actor. If none is given,
    +	 *                                      the actor will use a random name.
    +	 * @param leaderRetrievalService        Optionally, a leader retrieval service can be provided. If none is given,
    +	 *                                      then a LeaderRetrievalService is constructed from the configuration.
    +	 * @param localTaskManagerCommunication If true, the TaskManager will not initiate the TCP network stack.
    +	 * @param taskManagerClass      The class of the TaskManager actor. May be used to give
    +	 *                                      subclasses that understand additional actor messages.
    +	 * @return An ActorRef to the TaskManager actor.
    +	 * @throws org.apache.flink.configuration.IllegalConfigurationException     Thrown, if the given config contains illegal values.
    +	 * @throws java.io.IOException      Thrown, if any of the I/O components (such as buffer pools,
    +	 *                                       I/O manager, ...) cannot be properly started.
    +	 * @throws java.lang.Exception      Thrown is some other error occurs while parsing the configuration
    +	 *                                      or starting the TaskManager components.
    +	 */
    +	public static ActorRef startTaskManagerComponentsAndActor(
    +		Configuration configuration,
    +		ResourceID resourceID,
    +		ActorSystem actorSystem,
    +		String taskManagerHostname,
    +		String taskManagerActorName,
    +		LeaderRetrievalService leaderRetrievalService,
    +		boolean localTaskManagerCommunication,
    +		Class<? extends TaskManager> taskManagerClass) throws Exception {
    +
    +		Tuple4<TaskManagerConfiguration, NetworkEnvironmentConfiguration, InstanceConnectionInfo, MemoryType> tuple4
    +			= parseTaskManagerConfiguration(configuration, taskManagerHostname, localTaskManagerCommunication);
    +
    +		TaskManagerConfiguration taskManagerConfig = tuple4._1();
    +		NetworkEnvironmentConfiguration netConfig = tuple4._2();
    +		InstanceConnectionInfo connectionInfo = tuple4._3();
    +		MemoryType memType = tuple4._4();
    +
    +		// pre-start checks
    +		checkTempDirs(taskManagerConfig.tmpDirPaths());
    +
    +		ExecutionContext executionContext = ExecutionContexts$.MODULE$.fromExecutor(new ForkJoinPool());
    +
    +		// we start the network first, to make sure it can allocate its buffers first
    +		NetworkEnvironment network = new NetworkEnvironment(executionContext, taskManagerConfig.timeout(), netConfig);
    +
    +		// computing the amount of memory to use depends on how much memory is available
    +		// it strictly needs to happen AFTER the network stack has been initialized
    +
    +		// check if a value has been configured
    +		long configuredMemory = configuration.getLong(ConfigConstants.TASK_MANAGER_MEMORY_SIZE_KEY, -1L);
    +		checkConfigParameter(configuredMemory == -1 || configuredMemory > 0, configuredMemory,
    +			ConfigConstants.TASK_MANAGER_MEMORY_SIZE_KEY,
    +			"MemoryManager needs at least one MB of memory. " +
    +				"If you leave this config parameter empty, the system automatically " +
    +				"pick a fraction of the available memory.");
    +
    +		long memorySize;
    +		boolean preAllocateMemory = configuration.getBoolean(
    +			ConfigConstants.TASK_MANAGER_MEMORY_PRE_ALLOCATE_KEY,
    +			ConfigConstants.DEFAULT_TASK_MANAGER_MEMORY_PRE_ALLOCATE);
    +		if (configuredMemory > 0) {
    +			if (preAllocateMemory) {
    +				LOG.info("Using {} MB for managed memory." , configuredMemory);
    +			} else {
    +				LOG.info("Limiting managed memory to {} MB, memory will be allocated lazily." , configuredMemory);
    +			}
    +			memorySize = configuredMemory << 20; // megabytes to bytes
    +		} else {
    +			float fraction = configuration.getFloat(
    +				ConfigConstants.TASK_MANAGER_MEMORY_FRACTION_KEY,
    +				ConfigConstants.DEFAULT_MEMORY_MANAGER_MEMORY_FRACTION);
    +			checkConfigParameter(fraction > 0.0f && fraction < 1.0f, fraction,
    +				ConfigConstants.TASK_MANAGER_MEMORY_FRACTION_KEY,
    +				"MemoryManager fraction of the free memory must be between 0.0 and 1.0");
    +
    +			if (memType == MemoryType.HEAP) {
    +				long relativeMemSize = (long) (EnvironmentInformation.getSizeOfFreeHeapMemoryWithDefrag() * fraction);
    +				if (preAllocateMemory) {
    +					LOG.info("Using {} of the currently free heap space for managed heap memory ({} MB)." ,
    +						fraction , relativeMemSize >> 20);
    +				} else {
    +					LOG.info("Limiting managed memory to {} of the currently free heap space ({} MB), " +
    +						"memory will be allocated lazily." , fraction , relativeMemSize >> 20);
    +				}
    +				memorySize = relativeMemSize;
    +			} else if (memType == MemoryType.OFF_HEAP) {
    +				// The maximum heap memory has been adjusted according to the fraction
    +				long maxMemory = EnvironmentInformation.getMaxJvmHeapMemory();
    +				long directMemorySize = (long) (maxMemory / (1.0 - fraction) * fraction);
    +				if (preAllocateMemory) {
    +					LOG.info("Using {} of the maximum memory size for managed off-heap memory ({} MB)." ,
    +						fraction, directMemorySize >> 20);
    +				} else {
    +					LOG.info("Limiting managed memory to {} of the maximum memory size ({} MB)," +
    +						" memory will be allocated lazily.", fraction, directMemorySize >> 20);
    +				}
    +				memorySize = directMemorySize;
    +			} else {
    +				throw new RuntimeException("No supported memory type detected.");
    +			}
    +		}
    +
    +		// now start the memory manager
    +		MemoryManager memoryManager;
    +		try {
    +			memoryManager = new MemoryManager(
    +				memorySize,
    +				taskManagerConfig.numberOfSlots(),
    +				netConfig.networkBufferSize(),
    +				memType,
    +				preAllocateMemory);
    +		} catch (OutOfMemoryError e) {
    +			if (memType == MemoryType.HEAP) {
    +				throw new Exception("OutOfMemory error (" + e.getMessage() +
    +					") while allocating the TaskManager heap memory (" + memorySize + " bytes).", e);
    +			} else if (memType == MemoryType.OFF_HEAP) {
    +				throw new Exception("OutOfMemory error (" + e.getMessage() +
    +					") while allocating the TaskManager off-heap memory (" + memorySize +
    +					" bytes).Try increasing the maximum direct memory (-XX:MaxDirectMemorySize)", e);
    +			} else {
    +				throw e;
    +			}
    +		}
    +
    +		// start the I/O manager, it will create some temp directories.
    +		IOManager ioManager = new IOManagerAsync(taskManagerConfig.tmpDirPaths());
    +
    +		if (leaderRetrievalService == null){
    +			leaderRetrievalService = LeaderRetrievalUtils.createLeaderRetrievalService(configuration);
    +		}
    +
    +		// create the actor properties (which define the actor constructor parameters)
    +		Props tmProps = Props.create(
    +			taskManagerClass,
    +			taskManagerConfig,
    +			resourceID,
    +			connectionInfo,
    +			memoryManager,
    +			ioManager,
    +			network,
    +			taskManagerConfig.numberOfSlots(),
    +			leaderRetrievalService);
    --- End diff --
    
    You're still passing the `LeaderRetrievalService` which should not be used anymore.


---
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.
---

[GitHub] flink pull request #2400: [FLINK-4363] Implement TaskManager basic startup o...

Posted by mxm <gi...@git.apache.org>.
Github user mxm commented on a diff in the pull request:

    https://github.com/apache/flink/pull/2400#discussion_r75647940
  
    --- Diff: flink-runtime/src/main/java/org/apache/flink/runtime/rpc/taskexecutor/TaskExecutor.java ---
    @@ -35,27 +79,634 @@
      */
     public class TaskExecutor extends RpcEndpoint<TaskExecutorGateway> {
     
    -	/** The unique resource ID of this TaskExecutor */
    +	private static final Logger LOG = LoggerFactory.getLogger(TaskExecutor.class);
    +
    +	/** Return code for critical errors during the runtime */
    +	private static final int RUNTIME_FAILURE_RETURN_CODE = 2;
    +
    +	/** The name of the TaskManager actor */
    +	private static final String TASK_MANAGER_NAME = "taskmanager";
    +
    +	/** The unique resource ID of this TaskManager */
     	private final ResourceID resourceID;
     
     	/** The access to the leader election and metadata storage services */
     	private final HighAvailabilityServices haServices;
     
    -	// --------- resource manager --------
    +	/** The task manager configuration */
    +	private final TaskManagerConfiguration taskManagerConfig;
    +
    +	/** The connection information of the task manager */
    +	private final InstanceConnectionInfo connectionInfo;
    +
    +	/** The I/O manager component in the task manager */
    +	private final IOManager ioManager;
    +
    +	/** The memory manager component in the task manager */
    +	private final MemoryManager memoryManager;
    +
    +	/** The network component in the task manager */
    +	private final NetworkEnvironment networkEnvironment;
    +
    +	/** The number of slots in the task manager, should be 1 for YARN */
    +	private final int numberOfSlots;
     
    +	// --------- resource manager --------
     	private TaskExecutorToResourceManagerConnection resourceManagerConnection;
     
     	// ------------------------------------------------------------------------
     
     	public TaskExecutor(
    +			TaskManagerConfiguration taskManagerConfig,
    +			ResourceID resourceID,
    +			InstanceConnectionInfo connectionInfo,
    +			MemoryManager memoryManager,
    +			IOManager ioManager,
    +			NetworkEnvironment networkEnvironment,
    +			int numberOfSlots,
     			RpcService rpcService,
    -			HighAvailabilityServices haServices,
    -			ResourceID resourceID) {
    +			HighAvailabilityServices haServices) {
     
     		super(rpcService);
     
    -		this.haServices = checkNotNull(haServices);
    +		this.taskManagerConfig = checkNotNull(taskManagerConfig);
     		this.resourceID = checkNotNull(resourceID);
    +		this.connectionInfo = checkNotNull(connectionInfo);
    +		this.memoryManager = checkNotNull(memoryManager);
    +		this.ioManager = checkNotNull(ioManager);
    +		this.networkEnvironment = checkNotNull(networkEnvironment);
    +		this.numberOfSlots = checkNotNull(numberOfSlots);
    +		this.haServices = checkNotNull(haServices);
    +	}
    +
    +	/**
    +	 * Starts and runs the TaskManager.
    +	 * <p/>
    +	 * This method first tries to select the network interface to use for the TaskManager
    +	 * communication. The network interface is used both for the actor communication
    +	 * (coordination) as well as for the data exchange between task managers. Unless
    +	 * the hostname/interface is explicitly configured in the configuration, this
    +	 * method will try out various interfaces and methods to connect to the JobManager
    +	 * and select the one where the connection attempt is successful.
    +	 * <p/>
    +	 * After selecting the network interface, this method brings up an actor system
    +	 * for the TaskManager and its actors, starts the TaskManager's services
    +	 * (library cache, shuffle network stack, ...), and starts the TaskManager itself.
    +	 *
    +	 * @param configuration    The configuration for the TaskManager.
    +	 * @param taskManagerClass The actor class to instantiate.
    +	 *                         Allows to use TaskManager subclasses for example for YARN.
    +	 */
    +	public static void selectNetworkInterfaceAndRunTaskManager(
    +		Configuration configuration,
    +		ResourceID resourceID,
    +		Class<? extends TaskManager> taskManagerClass) throws Exception {
    +
    +		Tuple2<String, Integer> tuple2 = selectNetworkInterfaceAndPort(configuration);
    +
    +		runTaskManager(tuple2._1(), resourceID, tuple2._2(), configuration, taskManagerClass);
    +	}
    +
    +	private static Tuple2<String, Integer> selectNetworkInterfaceAndPort(Configuration configuration)
    +		throws Exception {
    +		String taskManagerHostname = configuration.getString(ConfigConstants.TASK_MANAGER_HOSTNAME_KEY, null);
    +		if (taskManagerHostname != null) {
    +			LOG.info("Using configured hostname/address for TaskManager: " + taskManagerHostname);
    +		} else {
    +			LeaderRetrievalService leaderRetrievalService = LeaderRetrievalUtils.createLeaderRetrievalService(configuration);
    +			FiniteDuration lookupTimeout = AkkaUtils.getLookupTimeout(configuration);
    +
    +			InetAddress taskManagerAddress = LeaderRetrievalUtils.findConnectingAddress(leaderRetrievalService, lookupTimeout);
    +			taskManagerHostname = taskManagerAddress.getHostName();
    +			LOG.info("TaskManager will use hostname/address '{}' ({}) for communication.",
    +				taskManagerHostname, taskManagerAddress.getHostAddress());
    +		}
    +
    +		// if no task manager port has been configured, use 0 (system will pick any free port)
    +		int actorSystemPort = configuration.getInteger(ConfigConstants.TASK_MANAGER_IPC_PORT_KEY, 0);
    +		if (actorSystemPort < 0 || actorSystemPort > 65535) {
    +			throw new IllegalConfigurationException("Invalid value for '" +
    +				ConfigConstants.TASK_MANAGER_IPC_PORT_KEY +
    +				"' (port for the TaskManager actor system) : " + actorSystemPort +
    +				" - Leave config parameter empty or use 0 to let the system choose a port automatically.");
    +		}
    +
    +		return new Tuple2<>(taskManagerHostname, actorSystemPort);
    +	}
    +
    +	/**
    +	 * Starts and runs the TaskManager. Brings up an actor system for the TaskManager and its
    +	 * actors, starts the TaskManager's services (library cache, shuffle network stack, ...),
    +	 * and starts the TaskManager itself.
    +	 * <p/>
    +	 * This method will also spawn a process reaper for the TaskManager (kill the process if
    +	 * the actor fails) and optionally start the JVM memory logging thread.
    +	 *
    +	 * @param taskManagerHostname The hostname/address of the interface where the actor system
    +	 *                            will communicate.
    +	 * @param resourceID          The id of the resource which the task manager will run on.
    +	 * @param actorSystemPort     The port at which the actor system will communicate.
    +	 * @param configuration       The configuration for the TaskManager.
    +	 * @param taskManagerClass    The actor class to instantiate. Allows the use of TaskManager
    +	 *                            subclasses for example for YARN.
    +	 */
    +	private static void runTaskManager(
    +		String taskManagerHostname,
    +		ResourceID resourceID,
    +		int actorSystemPort,
    +		Configuration configuration,
    +		Class<? extends TaskManager> taskManagerClass) throws Exception {
    +
    +		LOG.info("Starting TaskManager");
    +
    +		// Bring up the TaskManager actor system first, bind it to the given address.
    +
    +		LOG.info("Starting TaskManager actor system at " +
    +			NetUtils.hostAndPortToUrlString(taskManagerHostname, actorSystemPort));
    +
    +		ActorSystem taskManagerSystem;
    +		try {
    +			Tuple2<String, Object> address = new Tuple2<String, Object>(taskManagerHostname, actorSystemPort);
    +			Config akkaConfig = AkkaUtils.getAkkaConfig(configuration, new Some<>(address));
    +			LOG.debug("Using akka configuration\n " + akkaConfig);
    +			taskManagerSystem = AkkaUtils.createActorSystem(akkaConfig);
    +		} catch (Throwable t) {
    +			if (t instanceof org.jboss.netty.channel.ChannelException) {
    +				Throwable cause = t.getCause();
    +				if (cause != null && t.getCause() instanceof java.net.BindException) {
    +					String address = NetUtils.hostAndPortToUrlString(taskManagerHostname, actorSystemPort);
    +					throw new IOException("Unable to bind TaskManager actor system to address " +
    +						address + " - " + cause.getMessage(), t);
    +				}
    +			}
    +			throw new Exception("Could not create TaskManager actor system", t);
    +		}
    +
    +		// start all the TaskManager services (network stack,  library cache, ...)
    +		// and the TaskManager actor
    +		try {
    +			LOG.info("Starting TaskManager actor");
    +			ActorRef taskManagerActor = startTaskManagerComponentsAndActor(
    +				configuration,
    +				resourceID,
    +				taskManagerSystem,
    +				taskManagerHostname,
    +				TASK_MANAGER_NAME,
    +				null,
    +				false,
    +				taskManagerClass);
    +
    +			// start a process reaper that watches the JobManager. If the TaskManager actor dies,
    +			// the process reaper will kill the JVM process (to ensure easy failure detection)
    +			LOG.debug("Starting TaskManager process reaper");
    +
    +			taskManagerSystem.actorOf(
    +				Props.create(ProcessReaper.class, taskManagerActor, LOG, RUNTIME_FAILURE_RETURN_CODE),
    +				"TaskManager_Process_Reaper");
    +
    +			// if desired, start the logging daemon that periodically logs the memory usage information
    +			if (LOG.isInfoEnabled() && configuration.getBoolean(
    +				ConfigConstants.TASK_MANAGER_DEBUG_MEMORY_USAGE_START_LOG_THREAD,
    +				ConfigConstants.DEFAULT_TASK_MANAGER_DEBUG_MEMORY_USAGE_START_LOG_THREAD)) {
    +				LOG.info("Starting periodic memory usage logger");
    +
    +				long interval = configuration.getLong(
    +					ConfigConstants.TASK_MANAGER_DEBUG_MEMORY_USAGE_LOG_INTERVAL_MS,
    +					ConfigConstants.DEFAULT_TASK_MANAGER_DEBUG_MEMORY_USAGE_LOG_INTERVAL_MS);
    +
    +				MemoryLogger logger = new MemoryLogger(LOG, interval, taskManagerSystem);
    +				logger.start();
    +			}
    +
    +			// block until everything is done
    +			taskManagerSystem.awaitTermination();
    +		} catch (Throwable t) {
    +			LOG.error("Error while starting up taskManager", t);
    +			try {
    +				taskManagerSystem.shutdown();
    +			} catch (Throwable tt) {
    +				LOG.warn("Could not cleanly shut down actor system", tt);
    +			}
    +			throw t;
    +		}
    +	}
    +
    +	// --------------------------------------------------------------------------
    +	//  Starting and running the TaskManager
    +	// --------------------------------------------------------------------------
    +
    +	/**
    +	 * @param configuration                 The configuration for the TaskManager.
    +	 * @param resourceID                    The id of the resource which the task manager will run on.
    +	 * @param actorSystem                  The actor system that should run the TaskManager actor.
    +	 * @param taskManagerHostname       The hostname/address that describes the TaskManager's data location.
    +	 * @param taskManagerActorName      Optionally the name of the TaskManager actor. If none is given,
    +	 *                                      the actor will use a random name.
    +	 * @param leaderRetrievalService        Optionally, a leader retrieval service can be provided. If none is given,
    +	 *                                      then a LeaderRetrievalService is constructed from the configuration.
    +	 * @param localTaskManagerCommunication If true, the TaskManager will not initiate the TCP network stack.
    +	 * @param taskManagerClass      The class of the TaskManager actor. May be used to give
    +	 *                                      subclasses that understand additional actor messages.
    +	 * @return An ActorRef to the TaskManager actor.
    +	 * @throws org.apache.flink.configuration.IllegalConfigurationException     Thrown, if the given config contains illegal values.
    +	 * @throws java.io.IOException      Thrown, if any of the I/O components (such as buffer pools,
    +	 *                                       I/O manager, ...) cannot be properly started.
    +	 * @throws java.lang.Exception      Thrown is some other error occurs while parsing the configuration
    +	 *                                      or starting the TaskManager components.
    +	 */
    +	public static ActorRef startTaskManagerComponentsAndActor(
    +		Configuration configuration,
    +		ResourceID resourceID,
    +		ActorSystem actorSystem,
    +		String taskManagerHostname,
    +		String taskManagerActorName,
    +		LeaderRetrievalService leaderRetrievalService,
    +		boolean localTaskManagerCommunication,
    +		Class<? extends TaskManager> taskManagerClass) throws Exception {
    +
    +		Tuple4<TaskManagerConfiguration, NetworkEnvironmentConfiguration, InstanceConnectionInfo, MemoryType> tuple4
    +			= parseTaskManagerConfiguration(configuration, taskManagerHostname, localTaskManagerCommunication);
    +
    +		TaskManagerConfiguration taskManagerConfig = tuple4._1();
    +		NetworkEnvironmentConfiguration netConfig = tuple4._2();
    +		InstanceConnectionInfo connectionInfo = tuple4._3();
    +		MemoryType memType = tuple4._4();
    +
    +		// pre-start checks
    +		checkTempDirs(taskManagerConfig.tmpDirPaths());
    +
    +		ExecutionContext executionContext = ExecutionContexts$.MODULE$.fromExecutor(new ForkJoinPool());
    +
    +		// we start the network first, to make sure it can allocate its buffers first
    +		NetworkEnvironment network = new NetworkEnvironment(executionContext, taskManagerConfig.timeout(), netConfig);
    +
    +		// computing the amount of memory to use depends on how much memory is available
    +		// it strictly needs to happen AFTER the network stack has been initialized
    +
    +		// check if a value has been configured
    +		long configuredMemory = configuration.getLong(ConfigConstants.TASK_MANAGER_MEMORY_SIZE_KEY, -1L);
    +		checkConfigParameter(configuredMemory == -1 || configuredMemory > 0, configuredMemory,
    +			ConfigConstants.TASK_MANAGER_MEMORY_SIZE_KEY,
    +			"MemoryManager needs at least one MB of memory. " +
    +				"If you leave this config parameter empty, the system automatically " +
    +				"pick a fraction of the available memory.");
    +
    +		long memorySize;
    +		boolean preAllocateMemory = configuration.getBoolean(
    +			ConfigConstants.TASK_MANAGER_MEMORY_PRE_ALLOCATE_KEY,
    +			ConfigConstants.DEFAULT_TASK_MANAGER_MEMORY_PRE_ALLOCATE);
    +		if (configuredMemory > 0) {
    +			if (preAllocateMemory) {
    +				LOG.info("Using {} MB for managed memory." , configuredMemory);
    +			} else {
    +				LOG.info("Limiting managed memory to {} MB, memory will be allocated lazily." , configuredMemory);
    +			}
    +			memorySize = configuredMemory << 20; // megabytes to bytes
    +		} else {
    +			float fraction = configuration.getFloat(
    +				ConfigConstants.TASK_MANAGER_MEMORY_FRACTION_KEY,
    +				ConfigConstants.DEFAULT_MEMORY_MANAGER_MEMORY_FRACTION);
    +			checkConfigParameter(fraction > 0.0f && fraction < 1.0f, fraction,
    +				ConfigConstants.TASK_MANAGER_MEMORY_FRACTION_KEY,
    +				"MemoryManager fraction of the free memory must be between 0.0 and 1.0");
    +
    +			if (memType == MemoryType.HEAP) {
    +				long relativeMemSize = (long) (EnvironmentInformation.getSizeOfFreeHeapMemoryWithDefrag() * fraction);
    +				if (preAllocateMemory) {
    +					LOG.info("Using {} of the currently free heap space for managed heap memory ({} MB)." ,
    +						fraction , relativeMemSize >> 20);
    +				} else {
    +					LOG.info("Limiting managed memory to {} of the currently free heap space ({} MB), " +
    +						"memory will be allocated lazily." , fraction , relativeMemSize >> 20);
    +				}
    +				memorySize = relativeMemSize;
    +			} else if (memType == MemoryType.OFF_HEAP) {
    +				// The maximum heap memory has been adjusted according to the fraction
    +				long maxMemory = EnvironmentInformation.getMaxJvmHeapMemory();
    +				long directMemorySize = (long) (maxMemory / (1.0 - fraction) * fraction);
    +				if (preAllocateMemory) {
    +					LOG.info("Using {} of the maximum memory size for managed off-heap memory ({} MB)." ,
    +						fraction, directMemorySize >> 20);
    +				} else {
    +					LOG.info("Limiting managed memory to {} of the maximum memory size ({} MB)," +
    +						" memory will be allocated lazily.", fraction, directMemorySize >> 20);
    +				}
    +				memorySize = directMemorySize;
    +			} else {
    +				throw new RuntimeException("No supported memory type detected.");
    +			}
    +		}
    +
    +		// now start the memory manager
    +		MemoryManager memoryManager;
    +		try {
    +			memoryManager = new MemoryManager(
    +				memorySize,
    +				taskManagerConfig.numberOfSlots(),
    +				netConfig.networkBufferSize(),
    +				memType,
    +				preAllocateMemory);
    +		} catch (OutOfMemoryError e) {
    +			if (memType == MemoryType.HEAP) {
    +				throw new Exception("OutOfMemory error (" + e.getMessage() +
    +					") while allocating the TaskManager heap memory (" + memorySize + " bytes).", e);
    +			} else if (memType == MemoryType.OFF_HEAP) {
    +				throw new Exception("OutOfMemory error (" + e.getMessage() +
    +					") while allocating the TaskManager off-heap memory (" + memorySize +
    +					" bytes).Try increasing the maximum direct memory (-XX:MaxDirectMemorySize)", e);
    +			} else {
    +				throw e;
    +			}
    +		}
    +
    +		// start the I/O manager, it will create some temp directories.
    +		IOManager ioManager = new IOManagerAsync(taskManagerConfig.tmpDirPaths());
    +
    +		if (leaderRetrievalService == null){
    +			leaderRetrievalService = LeaderRetrievalUtils.createLeaderRetrievalService(configuration);
    +		}
    +
    +		// create the actor properties (which define the actor constructor parameters)
    +		Props tmProps = Props.create(
    +			taskManagerClass,
    +			taskManagerConfig,
    +			resourceID,
    +			connectionInfo,
    +			memoryManager,
    +			ioManager,
    +			network,
    +			taskManagerConfig.numberOfSlots(),
    +			leaderRetrievalService);
    +
    +		ActorRef taskManagerActorRef;
    +		if (taskManagerActorName != null && !taskManagerActorName.equals("")) {
    +			taskManagerActorRef = actorSystem.actorOf(tmProps, taskManagerActorName);
    +		} else {
    +			taskManagerActorRef = actorSystem.actorOf(tmProps);
    +		}
    +
    +		return taskManagerActorRef;
    +	}
    +
    +	// --------------------------------------------------------------------------
    +	//  Parsing and checking the TaskManager Configuration
    +	// --------------------------------------------------------------------------
    +
    +	/**
    +	 * Utility method to extract TaskManager config parameters from the configuration and to
    +	 * sanity check them.
    +	 *
    +	 * @param configuration                 The configuration.
    +	 * @param taskManagerHostname           The host name under which the TaskManager communicates.
    +	 * @param localTaskManagerCommunication True, to skip initializing the network stack.
    +	 *                                      Use only in cases where only one task manager runs.
    +	 * @return A tuple (TaskManagerConfiguration, network configuration,
    +	 * InstanceConnectionInfo, JobManager actor Akka URL).
    +	 */
    +	private static Tuple4<TaskManagerConfiguration, NetworkEnvironmentConfiguration, InstanceConnectionInfo, MemoryType>
    +		parseTaskManagerConfiguration(Configuration configuration, String taskManagerHostname, boolean localTaskManagerCommunication)
    +		throws Exception {
    +
    +		// ------- read values from the config and check them ---------
    +		//                      (a lot of them)
    +
    +		// ----> hosts / ports for communication and data exchange
    +
    +		int dataport = configuration.getInteger(ConfigConstants.TASK_MANAGER_DATA_PORT_KEY,
    +			ConfigConstants.DEFAULT_TASK_MANAGER_DATA_PORT);
    +		if (dataport == 0) {
    +			dataport = NetUtils.getAvailablePort();
    +		}
    +		checkConfigParameter(dataport > 0, dataport, ConfigConstants.TASK_MANAGER_DATA_PORT_KEY,
    +			"Leave config parameter empty or use 0 to let the system choose a port automatically.");
    +
    +		InetAddress taskManagerAddress = InetAddress.getByName(taskManagerHostname);
    +		InstanceConnectionInfo connectionInfo = new InstanceConnectionInfo(taskManagerAddress, dataport);
    +
    +		// ----> memory / network stack (shuffles/broadcasts), task slots, temp directories
    +
    +		// we need this because many configs have been written with a "-1" entry
    +		int slots = configuration.getInteger(ConfigConstants.TASK_MANAGER_NUM_TASK_SLOTS, 1);
    +		if (slots == -1) {
    +			slots = 1;
    +		}
    +		checkConfigParameter(slots >= 1, slots, ConfigConstants.TASK_MANAGER_NUM_TASK_SLOTS,
    +			"Number of task slots must be at least one.");
    +
    +		int numNetworkBuffers = configuration.getInteger(
    +			ConfigConstants.TASK_MANAGER_NETWORK_NUM_BUFFERS_KEY,
    +			ConfigConstants.DEFAULT_TASK_MANAGER_NETWORK_NUM_BUFFERS);
    +		checkConfigParameter(numNetworkBuffers > 0, numNetworkBuffers,
    +			ConfigConstants.TASK_MANAGER_NETWORK_NUM_BUFFERS_KEY, "");
    +
    +		int pageSize = configuration.getInteger(
    +			ConfigConstants.TASK_MANAGER_MEMORY_SEGMENT_SIZE_KEY,
    +			ConfigConstants.DEFAULT_TASK_MANAGER_MEMORY_SEGMENT_SIZE);
    +		// check page size of for minimum size
    +		checkConfigParameter(pageSize >= MemoryManager.MIN_PAGE_SIZE, pageSize,
    +			ConfigConstants.TASK_MANAGER_MEMORY_SEGMENT_SIZE_KEY,
    +			"Minimum memory segment size is " + MemoryManager.MIN_PAGE_SIZE);
    +		// check page size for power of two
    +		checkConfigParameter(MathUtils.isPowerOf2(pageSize), pageSize,
    +			ConfigConstants.TASK_MANAGER_MEMORY_SEGMENT_SIZE_KEY,
    +			"Memory segment size must be a power of 2.");
    +
    +		// check whether we use heap or off-heap memory
    +		MemoryType memType;
    +		if (configuration.getBoolean(ConfigConstants.TASK_MANAGER_MEMORY_OFF_HEAP_KEY, false)) {
    +			memType = MemoryType.OFF_HEAP;
    +		} else {
    +			memType = MemoryType.HEAP;
    +		}
    +
    +		// initialize the memory segment factory accordingly
    +		if (memType == MemoryType.HEAP) {
    +			if (!MemorySegmentFactory.isInitialized()) {
    +				MemorySegmentFactory.initializeFactory(HeapMemorySegment.FACTORY);
    +			} else if (MemorySegmentFactory.getFactory() != HeapMemorySegment.FACTORY) {
    +				throw new Exception("Memory type is set to heap memory, but memory segment " +
    +					"factory has been initialized for off-heap memory segments");
    +			}
    +		} else {
    +			if (!MemorySegmentFactory.isInitialized()) {
    +				MemorySegmentFactory.initializeFactory(HybridMemorySegment.FACTORY);
    +			} else if (MemorySegmentFactory.getFactory() != HybridMemorySegment.FACTORY) {
    +				throw new Exception("Memory type is set to off-heap memory, but memory segment " +
    +					"factory has been initialized for heap memory segments");
    +			}
    +		}
    +
    +		String[] tmpDirs = configuration.getString(
    +			ConfigConstants.TASK_MANAGER_TMP_DIR_KEY,
    +			ConfigConstants.DEFAULT_TASK_MANAGER_TMP_PATH).split(",|" + File.pathSeparator);
    +
    +		NettyConfig nettyConfig = null;
    +		if (!localTaskManagerCommunication) {
    +			nettyConfig = new NettyConfig(connectionInfo.address(), connectionInfo.dataPort(), pageSize, slots, configuration);
    +		}
    +
    +		// Default spill I/O mode for intermediate results
    +		String syncOrAsync = configuration.getString(
    +			ConfigConstants.TASK_MANAGER_NETWORK_DEFAULT_IO_MODE,
    +			ConfigConstants.DEFAULT_TASK_MANAGER_NETWORK_DEFAULT_IO_MODE);
    +
    +		IOMode ioMode;
    +		if (syncOrAsync == "async") {
    +			ioMode = IOManager.IOMode.ASYNC;
    +		} else {
    +			ioMode = IOManager.IOMode.SYNC;
    +		}
    +
    +		NetworkEnvironmentConfiguration networkConfig = new NetworkEnvironmentConfiguration(
    +			numNetworkBuffers,
    +			pageSize,
    +			memType,
    +			ioMode,
    +			nettyConfig == null ? Option.<NettyConfig>empty() : new Some<>(nettyConfig),
    +			new Tuple2<>(500, 3000));
    +
    +		// ----> timeouts, library caching, profiling
    +
    +		FiniteDuration timeout;
    +		try {
    +			timeout = AkkaUtils.getTimeout(configuration);
    +		} catch (Exception e) {
    +			throw new IllegalArgumentException(
    +				"Invalid format for '" + ConfigConstants.AKKA_ASK_TIMEOUT +
    +					"'.Use formats like '50 s' or '1 min' to specify the timeout.");
    +		}
    +		LOG.info("Messages between TaskManager and JobManager have a max timeout of " + timeout);
    +
    +		long cleanupInterval = configuration.getLong(
    +			ConfigConstants.LIBRARY_CACHE_MANAGER_CLEANUP_INTERVAL,
    +			ConfigConstants.DEFAULT_LIBRARY_CACHE_MANAGER_CLEANUP_INTERVAL) * 1000;
    +
    +		FiniteDuration finiteRegistrationDuration = null;
    +		try {
    +			Duration maxRegistrationDuration = Duration.create(configuration.getString(
    +				ConfigConstants.TASK_MANAGER_MAX_REGISTRATION_DURATION,
    +				ConfigConstants.DEFAULT_TASK_MANAGER_MAX_REGISTRATION_DURATION));
    +			if (maxRegistrationDuration.isFinite()) {
    +				finiteRegistrationDuration = new FiniteDuration(maxRegistrationDuration.toSeconds(), TimeUnit.SECONDS);
    +			}
    +		} catch (NumberFormatException e) {
    +			throw new IllegalArgumentException("Invalid format for parameter " +
    +				ConfigConstants.TASK_MANAGER_MAX_REGISTRATION_DURATION, e);
    +		}
    +
    +		FiniteDuration initialRegistrationPause;
    +		try {
    +			Duration pause = Duration.create(configuration.getString(
    +				ConfigConstants.TASK_MANAGER_INITIAL_REGISTRATION_PAUSE,
    +				ConfigConstants.DEFAULT_TASK_MANAGER_INITIAL_REGISTRATION_PAUSE));
    +			if (pause.isFinite()) {
    +				initialRegistrationPause = new FiniteDuration(pause.toSeconds(), TimeUnit.SECONDS);
    +			} else {
    +				throw new IllegalArgumentException("The initial registration pause must be finite: " + pause);
    +			}
    +		} catch (NumberFormatException e) {
    +			throw new IllegalArgumentException("Invalid format for parameter " +
    +				ConfigConstants.TASK_MANAGER_INITIAL_REGISTRATION_PAUSE, e);
    +		}
    +
    +		FiniteDuration maxRegistrationPause;
    --- End diff --
    
    `final`


---
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.
---

[GitHub] flink pull request #2400: [FLINK-4363] Implement TaskManager basic startup o...

Posted by wangzhijiang999 <gi...@git.apache.org>.
Github user wangzhijiang999 commented on a diff in the pull request:

    https://github.com/apache/flink/pull/2400#discussion_r76190061
  
    --- Diff: flink-runtime/src/main/java/org/apache/flink/runtime/rpc/taskexecutor/TaskExecutor.java ---
    @@ -36,27 +82,617 @@
      */
     public class TaskExecutor extends RpcEndpoint<TaskExecutorGateway> {
     
    -	/** The unique resource ID of this TaskExecutor */
    +	private static final Logger LOG = LoggerFactory.getLogger(TaskExecutor.class);
    +
    +	/** The unique resource ID of this TaskManager */
     	private final ResourceID resourceID;
     
     	/** The access to the leader election and metadata storage services */
     	private final HighAvailabilityServices haServices;
     
    -	// --------- resource manager --------
    +	/** The task manager configuration */
    +	private final TaskExecutorConfiguration taskExecutorConfig;
    +
    +	/** The connection information of the task manager */
    +	private final InstanceConnectionInfo connectionInfo;
    +
    +	/** The I/O manager component in the task manager */
    +	private final IOManager ioManager;
    +
    +	/** The memory manager component in the task manager */
    +	private final MemoryManager memoryManager;
    +
    +	/** The network component in the task manager */
    +	private final NetworkEnvironment networkEnvironment;
    +
    +	/** The number of slots in the task manager, should be 1 for YARN */
    +	private final int numberOfSlots;
     
    +	// --------- resource manager --------
     	private TaskExecutorToResourceManagerConnection resourceManagerConnection;
     
     	// ------------------------------------------------------------------------
     
     	public TaskExecutor(
    +			TaskExecutorConfiguration taskExecutorConfig,
    +			ResourceID resourceID,
    +			InstanceConnectionInfo connectionInfo,
    +			MemoryManager memoryManager,
    +			IOManager ioManager,
    +			NetworkEnvironment networkEnvironment,
    +			int numberOfSlots,
     			RpcService rpcService,
    -			HighAvailabilityServices haServices,
    -			ResourceID resourceID) {
    +			HighAvailabilityServices haServices) {
     
     		super(rpcService);
     
    -		this.haServices = checkNotNull(haServices);
    +		this.taskExecutorConfig = checkNotNull(taskExecutorConfig);
     		this.resourceID = checkNotNull(resourceID);
    +		this.connectionInfo = checkNotNull(connectionInfo);
    +		this.memoryManager = checkNotNull(memoryManager);
    +		this.ioManager = checkNotNull(ioManager);
    +		this.networkEnvironment = checkNotNull(networkEnvironment);
    +		this.numberOfSlots = checkNotNull(numberOfSlots);
    +		this.haServices = checkNotNull(haServices);
    +	}
    +
    +	/**
    +	 * Starts and runs the TaskManager.
    +	 * <p/>
    +	 * This method first tries to select the network interface to use for the TaskManager
    +	 * communication. The network interface is used both for the actor communication
    +	 * (coordination) as well as for the data exchange between task managers. Unless
    +	 * the hostname/interface is explicitly configured in the configuration, this
    +	 * method will try out various interfaces and methods to connect to the JobManager
    +	 * and select the one where the connection attempt is successful.
    +	 * <p/>
    +	 * After selecting the network interface, this method brings up an actor system
    +	 * for the TaskManager and its actors, starts the TaskManager's services
    +	 * (library cache, shuffle network stack, ...), and starts the TaskManager itself.
    +	 *
    +	 * @param configuration    The configuration for the TaskManager.
    +	 * @param resourceID       The id of the resource which the task manager will run on.
    +	 */
    +	public static void selectNetworkInterfaceAndRunTaskManager(
    +		Configuration configuration,
    +		ResourceID resourceID) throws Exception {
    +
    +		InetSocketAddress taskManagerAddress = selectNetworkInterfaceAndPort(configuration);
    +
    +		runTaskManager(taskManagerAddress.getHostName(), resourceID, taskManagerAddress.getPort(), configuration);
    +	}
    +
    +	private static InetSocketAddress selectNetworkInterfaceAndPort(Configuration configuration)
    +		throws Exception {
    +		String taskManagerHostname = configuration.getString(ConfigConstants.TASK_MANAGER_HOSTNAME_KEY, null);
    +		if (taskManagerHostname != null) {
    +			LOG.info("Using configured hostname/address for TaskManager: " + taskManagerHostname);
    +		} else {
    +			LeaderRetrievalService leaderRetrievalService = LeaderRetrievalUtils.createLeaderRetrievalService(configuration);
    +			FiniteDuration lookupTimeout = AkkaUtils.getLookupTimeout(configuration);
    +
    +			InetAddress taskManagerAddress = LeaderRetrievalUtils.findConnectingAddress(leaderRetrievalService, lookupTimeout);
    +			taskManagerHostname = taskManagerAddress.getHostName();
    +			LOG.info("TaskManager will use hostname/address '{}' ({}) for communication.",
    +				taskManagerHostname, taskManagerAddress.getHostAddress());
    +		}
    +
    +		// if no task manager port has been configured, use 0 (system will pick any free port)
    +		int actorSystemPort = configuration.getInteger(ConfigConstants.TASK_MANAGER_IPC_PORT_KEY, 0);
    +		if (actorSystemPort < 0 || actorSystemPort > 65535) {
    +			throw new IllegalConfigurationException("Invalid value for '" +
    +				ConfigConstants.TASK_MANAGER_IPC_PORT_KEY +
    +				"' (port for the TaskManager actor system) : " + actorSystemPort +
    +				" - Leave config parameter empty or use 0 to let the system choose a port automatically.");
    +		}
    +
    +		return new InetSocketAddress(taskManagerHostname, actorSystemPort);
    +	}
    +
    +	/**
    +	 * Starts and runs the TaskManager. Brings up an actor system for the TaskManager and its
    +	 * actors, starts the TaskManager's services (library cache, shuffle network stack, ...),
    +	 * and starts the TaskManager itself.
    +	 * <p/>
    +	 * This method will also spawn a process reaper for the TaskManager (kill the process if
    +	 * the actor fails) and optionally start the JVM memory logging thread.
    +	 *
    +	 * @param taskManagerHostname The hostname/address of the interface where the actor system
    +	 *                            will communicate.
    +	 * @param resourceID          The id of the resource which the task manager will run on.
    +	 * @param actorSystemPort   The port at which the actor system will communicate.
    +	 * @param configuration       The configuration for the TaskManager.
    +	 */
    +	private static void runTaskManager(
    +		String taskManagerHostname,
    +		ResourceID resourceID,
    +		int actorSystemPort,
    +		final Configuration configuration) throws Exception {
    +
    +		LOG.info("Starting TaskManager");
    +
    +		// Bring up the TaskManager actor system first, bind it to the given address.
    +
    +		LOG.info("Starting TaskManager actor system at " +
    +			NetUtils.hostAndPortToUrlString(taskManagerHostname, actorSystemPort));
    +
    +		final ActorSystem taskManagerSystem;
    +		try {
    +			Tuple2<String, Object> address = new Tuple2<String, Object>(taskManagerHostname, actorSystemPort);
    +			Config akkaConfig = AkkaUtils.getAkkaConfig(configuration, new Some<>(address));
    +			LOG.debug("Using akka configuration\n " + akkaConfig);
    +			taskManagerSystem = AkkaUtils.createActorSystem(akkaConfig);
    +		} catch (Throwable t) {
    +			if (t instanceof org.jboss.netty.channel.ChannelException) {
    +				Throwable cause = t.getCause();
    +				if (cause != null && t.getCause() instanceof java.net.BindException) {
    +					String address = NetUtils.hostAndPortToUrlString(taskManagerHostname, actorSystemPort);
    +					throw new IOException("Unable to bind TaskManager actor system to address " +
    +						address + " - " + cause.getMessage(), t);
    +				}
    +			}
    +			throw new Exception("Could not create TaskManager actor system", t);
    +		}
    +
    +		// start akka rpc service based on actor system
    +		Timeout timeout = new Timeout(AkkaUtils.getTimeout(configuration).toMillis(), TimeUnit.MILLISECONDS);
    +		AkkaRpcService akkaRpcService = new AkkaRpcService(taskManagerSystem, timeout);
    +
    +		// start high availability service to implement getResourceManagerLeaderRetriever method only
    +		HighAvailabilityServices haServices = new HighAvailabilityServices() {
    +			@Override
    +			public LeaderRetrievalService getResourceManagerLeaderRetriever() throws Exception {
    +				return LeaderRetrievalUtils.createLeaderRetrievalService(configuration);
    +			}
    +
    +			@Override
    +			public LeaderElectionService getJobMasterLeaderElectionService(JobID jobID) throws Exception {
    +				return null;
    +			}
    +		};
    +
    +		// start all the TaskManager services (network stack,  library cache, ...)
    +		// and the TaskManager actor
    +		try {
    +			LOG.info("Starting TaskManager actor");
    +			TaskExecutor taskExecutor = startTaskManagerComponentsAndActor(
    +				configuration,
    +				resourceID,
    +				akkaRpcService,
    +				taskManagerHostname,
    +				haServices,
    +				false);
    +
    +			taskExecutor.start();
    +
    +			// if desired, start the logging daemon that periodically logs the memory usage information
    +			if (LOG.isInfoEnabled() && configuration.getBoolean(
    +				ConfigConstants.TASK_MANAGER_DEBUG_MEMORY_USAGE_START_LOG_THREAD,
    +				ConfigConstants.DEFAULT_TASK_MANAGER_DEBUG_MEMORY_USAGE_START_LOG_THREAD)) {
    +				LOG.info("Starting periodic memory usage logger");
    +
    +				long interval = configuration.getLong(
    +					ConfigConstants.TASK_MANAGER_DEBUG_MEMORY_USAGE_LOG_INTERVAL_MS,
    +					ConfigConstants.DEFAULT_TASK_MANAGER_DEBUG_MEMORY_USAGE_LOG_INTERVAL_MS);
    +
    +				MemoryLogger logger = new MemoryLogger(LOG, interval, taskManagerSystem);
    +				logger.start();
    +			}
    +
    +			// block until everything is done
    +			taskManagerSystem.awaitTermination();
    +		} catch (Throwable t) {
    +			LOG.error("Error while starting up taskManager", t);
    +			try {
    +				taskManagerSystem.shutdown();
    +			} catch (Throwable tt) {
    +				LOG.warn("Could not cleanly shut down actor system", tt);
    +			}
    +			throw t;
    +		}
    +	}
    +
    +	// --------------------------------------------------------------------------
    +	//  Starting and running the TaskManager
    +	// --------------------------------------------------------------------------
    +
    +	/**
    +	 * @param configuration                 The configuration for the TaskManager.
    +	 * @param resourceID                    The id of the resource which the task manager will run on.
    +	 * @param rpcService                  The rpc service which is used to start and connect to the TaskManager RpcEndpoint .
    +	 * @param taskManagerHostname       The hostname/address that describes the TaskManager's data location.
    +	 * @param haServices        Optionally, a high availability service can be provided. If none is given,
    +	 *                                      then a HighAvailabilityServices is constructed from the configuration.
    +	 * @param localTaskManagerCommunication If true, the TaskManager will not initiate the TCP network stack.
    +	 * @return An ActorRef to the TaskManager actor.
    +	 * @throws org.apache.flink.configuration.IllegalConfigurationException     Thrown, if the given config contains illegal values.
    +	 * @throws java.io.IOException      Thrown, if any of the I/O components (such as buffer pools,
    +	 *                                       I/O manager, ...) cannot be properly started.
    +	 * @throws java.lang.Exception      Thrown is some other error occurs while parsing the configuration
    +	 *                                      or starting the TaskManager components.
    +	 */
    +	public static TaskExecutor startTaskManagerComponentsAndActor(
    +		Configuration configuration,
    +		ResourceID resourceID,
    +		RpcService rpcService,
    +		String taskManagerHostname,
    +		HighAvailabilityServices haServices,
    +		boolean localTaskManagerCommunication) throws Exception {
    +
    +		Tuple4<TaskExecutorConfiguration, NetworkEnvironmentConfiguration, InstanceConnectionInfo, MemoryType> tuple4
    +			= parseTaskManagerConfiguration(configuration, taskManagerHostname, localTaskManagerCommunication);
    +
    +		TaskExecutorConfiguration taskExecutorConfig = tuple4.f0;
    +		NetworkEnvironmentConfiguration netConfig = tuple4.f1;
    +		InstanceConnectionInfo connectionInfo = tuple4.f2;
    +		MemoryType memType = tuple4.f3;
    +
    +		// pre-start checks
    +		checkTempDirs(taskExecutorConfig.getTmpDirPaths());
    +
    +		ExecutionContext executionContext = ExecutionContexts$.MODULE$.fromExecutor(new ForkJoinPool());
    +
    +		// we start the network first, to make sure it can allocate its buffers first
    +		NetworkEnvironment network = new NetworkEnvironment(executionContext, taskExecutorConfig.getTimeout(), netConfig);
    +
    +		// computing the amount of memory to use depends on how much memory is available
    +		// it strictly needs to happen AFTER the network stack has been initialized
    +
    +		// check if a value has been configured
    +		long configuredMemory = configuration.getLong(ConfigConstants.TASK_MANAGER_MEMORY_SIZE_KEY, -1L);
    +		checkConfigParameter(configuredMemory == -1 || configuredMemory > 0, configuredMemory,
    +			ConfigConstants.TASK_MANAGER_MEMORY_SIZE_KEY,
    +			"MemoryManager needs at least one MB of memory. " +
    +				"If you leave this config parameter empty, the system automatically " +
    +				"pick a fraction of the available memory.");
    +
    +		long memorySize;
    +		boolean preAllocateMemory = configuration.getBoolean(
    +			ConfigConstants.TASK_MANAGER_MEMORY_PRE_ALLOCATE_KEY,
    +			ConfigConstants.DEFAULT_TASK_MANAGER_MEMORY_PRE_ALLOCATE);
    +		if (configuredMemory > 0) {
    +			if (preAllocateMemory) {
    +				LOG.info("Using {} MB for managed memory." , configuredMemory);
    +			} else {
    +				LOG.info("Limiting managed memory to {} MB, memory will be allocated lazily." , configuredMemory);
    +			}
    +			memorySize = configuredMemory << 20; // megabytes to bytes
    +		} else {
    +			float fraction = configuration.getFloat(
    +				ConfigConstants.TASK_MANAGER_MEMORY_FRACTION_KEY,
    +				ConfigConstants.DEFAULT_MEMORY_MANAGER_MEMORY_FRACTION);
    +			checkConfigParameter(fraction > 0.0f && fraction < 1.0f, fraction,
    +				ConfigConstants.TASK_MANAGER_MEMORY_FRACTION_KEY,
    +				"MemoryManager fraction of the free memory must be between 0.0 and 1.0");
    +
    +			if (memType == MemoryType.HEAP) {
    +				long relativeMemSize = (long) (EnvironmentInformation.getSizeOfFreeHeapMemoryWithDefrag() * fraction);
    +				if (preAllocateMemory) {
    +					LOG.info("Using {} of the currently free heap space for managed heap memory ({} MB)." ,
    +						fraction , relativeMemSize >> 20);
    +				} else {
    +					LOG.info("Limiting managed memory to {} of the currently free heap space ({} MB), " +
    +						"memory will be allocated lazily." , fraction , relativeMemSize >> 20);
    +				}
    +				memorySize = relativeMemSize;
    +			} else if (memType == MemoryType.OFF_HEAP) {
    +				// The maximum heap memory has been adjusted according to the fraction
    +				long maxMemory = EnvironmentInformation.getMaxJvmHeapMemory();
    +				long directMemorySize = (long) (maxMemory / (1.0 - fraction) * fraction);
    +				if (preAllocateMemory) {
    +					LOG.info("Using {} of the maximum memory size for managed off-heap memory ({} MB)." ,
    +						fraction, directMemorySize >> 20);
    +				} else {
    +					LOG.info("Limiting managed memory to {} of the maximum memory size ({} MB)," +
    +						" memory will be allocated lazily.", fraction, directMemorySize >> 20);
    +				}
    +				memorySize = directMemorySize;
    +			} else {
    +				throw new RuntimeException("No supported memory type detected.");
    +			}
    +		}
    +
    +		// now start the memory manager
    +		final MemoryManager memoryManager;
    +		try {
    +			memoryManager = new MemoryManager(
    +				memorySize,
    +				taskExecutorConfig.getNumberOfSlots(),
    +				netConfig.networkBufferSize(),
    +				memType,
    +				preAllocateMemory);
    +		} catch (OutOfMemoryError e) {
    +			if (memType == MemoryType.HEAP) {
    +				throw new Exception("OutOfMemory error (" + e.getMessage() +
    +					") while allocating the TaskManager heap memory (" + memorySize + " bytes).", e);
    +			} else if (memType == MemoryType.OFF_HEAP) {
    +				throw new Exception("OutOfMemory error (" + e.getMessage() +
    +					") while allocating the TaskManager off-heap memory (" + memorySize +
    +					" bytes).Try increasing the maximum direct memory (-XX:MaxDirectMemorySize)", e);
    +			} else {
    +				throw e;
    +			}
    +		}
    +
    +		// start the I/O manager, it will create some temp directories.
    +		IOManager ioManager = new IOManagerAsync(taskExecutorConfig.getTmpDirPaths());
    +
    +		TaskExecutor taskExecutor = new TaskExecutor(
    +			taskExecutorConfig,
    +			resourceID,
    +			connectionInfo,
    +			memoryManager,
    +			ioManager,
    +			network,
    +			taskExecutorConfig.getNumberOfSlots(),
    +			rpcService,
    +			haServices);
    +
    +		return taskExecutor;
    +	}
    +
    +	// --------------------------------------------------------------------------
    +	//  Parsing and checking the TaskManager Configuration
    +	// --------------------------------------------------------------------------
    +
    +	/**
    +	 * Utility method to extract TaskManager config parameters from the configuration and to
    +	 * sanity check them.
    +	 *
    +	 * @param configuration                 The configuration.
    +	 * @param taskManagerHostname           The host name under which the TaskManager communicates.
    +	 * @param localTaskManagerCommunication True, to skip initializing the network stack.
    +	 *                                      Use only in cases where only one task manager runs.
    +	 * @return A tuple (TaskManagerConfiguration, network configuration,
    +	 * InstanceConnectionInfo, JobManager actor Akka URL).
    +	 */
    +	private static Tuple4<TaskExecutorConfiguration, NetworkEnvironmentConfiguration, InstanceConnectionInfo, MemoryType>
    +		parseTaskManagerConfiguration(Configuration configuration, String taskManagerHostname, boolean localTaskManagerCommunication)
    +		throws Exception {
    +
    +		// ------- read values from the config and check them ---------
    +		//                      (a lot of them)
    +
    +		// ----> hosts / ports for communication and data exchange
    +
    +		int dataport = configuration.getInteger(ConfigConstants.TASK_MANAGER_DATA_PORT_KEY,
    +			ConfigConstants.DEFAULT_TASK_MANAGER_DATA_PORT);
    +		if (dataport == 0) {
    +			dataport = NetUtils.getAvailablePort();
    +		}
    +		checkConfigParameter(dataport > 0, dataport, ConfigConstants.TASK_MANAGER_DATA_PORT_KEY,
    +			"Leave config parameter empty or use 0 to let the system choose a port automatically.");
    +
    +		InetAddress taskManagerAddress = InetAddress.getByName(taskManagerHostname);
    +		InstanceConnectionInfo connectionInfo = new InstanceConnectionInfo(taskManagerAddress, dataport);
    +
    +		// ----> memory / network stack (shuffles/broadcasts), task slots, temp directories
    +
    +		// we need this because many configs have been written with a "-1" entry
    +		int slots = configuration.getInteger(ConfigConstants.TASK_MANAGER_NUM_TASK_SLOTS, 1);
    +		if (slots == -1) {
    +			slots = 1;
    +		}
    +		checkConfigParameter(slots >= 1, slots, ConfigConstants.TASK_MANAGER_NUM_TASK_SLOTS,
    +			"Number of task slots must be at least one.");
    +
    +		int numNetworkBuffers = configuration.getInteger(
    +			ConfigConstants.TASK_MANAGER_NETWORK_NUM_BUFFERS_KEY,
    +			ConfigConstants.DEFAULT_TASK_MANAGER_NETWORK_NUM_BUFFERS);
    +		checkConfigParameter(numNetworkBuffers > 0, numNetworkBuffers,
    +			ConfigConstants.TASK_MANAGER_NETWORK_NUM_BUFFERS_KEY, "");
    +
    +		int pageSize = configuration.getInteger(
    +			ConfigConstants.TASK_MANAGER_MEMORY_SEGMENT_SIZE_KEY,
    +			ConfigConstants.DEFAULT_TASK_MANAGER_MEMORY_SEGMENT_SIZE);
    +		// check page size of for minimum size
    +		checkConfigParameter(pageSize >= MemoryManager.MIN_PAGE_SIZE, pageSize,
    +			ConfigConstants.TASK_MANAGER_MEMORY_SEGMENT_SIZE_KEY,
    +			"Minimum memory segment size is " + MemoryManager.MIN_PAGE_SIZE);
    +		// check page size for power of two
    +		checkConfigParameter(MathUtils.isPowerOf2(pageSize), pageSize,
    +			ConfigConstants.TASK_MANAGER_MEMORY_SEGMENT_SIZE_KEY,
    +			"Memory segment size must be a power of 2.");
    +
    +		// check whether we use heap or off-heap memory
    +		final MemoryType memType;
    +		if (configuration.getBoolean(ConfigConstants.TASK_MANAGER_MEMORY_OFF_HEAP_KEY, false)) {
    +			memType = MemoryType.OFF_HEAP;
    +		} else {
    +			memType = MemoryType.HEAP;
    +		}
    +
    +		// initialize the memory segment factory accordingly
    +		if (memType == MemoryType.HEAP) {
    +			if (!MemorySegmentFactory.isInitialized()) {
    +				MemorySegmentFactory.initializeFactory(HeapMemorySegment.FACTORY);
    +			} else if (MemorySegmentFactory.getFactory() != HeapMemorySegment.FACTORY) {
    +				throw new Exception("Memory type is set to heap memory, but memory segment " +
    +					"factory has been initialized for off-heap memory segments");
    +			}
    +		} else {
    +			if (!MemorySegmentFactory.isInitialized()) {
    +				MemorySegmentFactory.initializeFactory(HybridMemorySegment.FACTORY);
    +			} else if (MemorySegmentFactory.getFactory() != HybridMemorySegment.FACTORY) {
    +				throw new Exception("Memory type is set to off-heap memory, but memory segment " +
    +					"factory has been initialized for heap memory segments");
    +			}
    +		}
    +
    +		String[] tmpDirs = configuration.getString(
    +			ConfigConstants.TASK_MANAGER_TMP_DIR_KEY,
    +			ConfigConstants.DEFAULT_TASK_MANAGER_TMP_PATH).split(",|" + File.pathSeparator);
    +
    +		final NettyConfig nettyConfig;
    +		if (!localTaskManagerCommunication) {
    +			nettyConfig = new NettyConfig(connectionInfo.address(), connectionInfo.dataPort(), pageSize, slots, configuration);
    +		} else {
    +			nettyConfig = null;
    +		}
    +
    +		// Default spill I/O mode for intermediate results
    +		String syncOrAsync = configuration.getString(
    +			ConfigConstants.TASK_MANAGER_NETWORK_DEFAULT_IO_MODE,
    +			ConfigConstants.DEFAULT_TASK_MANAGER_NETWORK_DEFAULT_IO_MODE);
    +
    +		final IOMode ioMode;
    +		if (syncOrAsync == "async") {
    --- End diff --
    
    Yes, it is a low level mistake. I forgot to translate the scala here.


---
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.
---

[GitHub] flink issue #2400: [FLINK-4363] Implement TaskManager basic startup of all c...

Posted by mxm <gi...@git.apache.org>.
Github user mxm commented on the issue:

    https://github.com/apache/flink/pull/2400
  
    Thank you for the pull request @wangzhijiang999! Looks good. I just had some minor remarks on the Scala-->Java conversion.


---
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.
---

[GitHub] flink pull request #2400: [FLINK-4363] Implement TaskManager basic startup o...

Posted by mxm <gi...@git.apache.org>.
Github user mxm commented on a diff in the pull request:

    https://github.com/apache/flink/pull/2400#discussion_r75645973
  
    --- Diff: flink-runtime/src/main/java/org/apache/flink/runtime/rpc/taskexecutor/TaskExecutor.java ---
    @@ -35,27 +79,634 @@
      */
     public class TaskExecutor extends RpcEndpoint<TaskExecutorGateway> {
     
    -	/** The unique resource ID of this TaskExecutor */
    +	private static final Logger LOG = LoggerFactory.getLogger(TaskExecutor.class);
    +
    +	/** Return code for critical errors during the runtime */
    +	private static final int RUNTIME_FAILURE_RETURN_CODE = 2;
    +
    +	/** The name of the TaskManager actor */
    +	private static final String TASK_MANAGER_NAME = "taskmanager";
    +
    +	/** The unique resource ID of this TaskManager */
     	private final ResourceID resourceID;
     
     	/** The access to the leader election and metadata storage services */
     	private final HighAvailabilityServices haServices;
     
    -	// --------- resource manager --------
    +	/** The task manager configuration */
    +	private final TaskManagerConfiguration taskManagerConfig;
    +
    +	/** The connection information of the task manager */
    +	private final InstanceConnectionInfo connectionInfo;
    +
    +	/** The I/O manager component in the task manager */
    +	private final IOManager ioManager;
    +
    +	/** The memory manager component in the task manager */
    +	private final MemoryManager memoryManager;
    +
    +	/** The network component in the task manager */
    +	private final NetworkEnvironment networkEnvironment;
    +
    +	/** The number of slots in the task manager, should be 1 for YARN */
    +	private final int numberOfSlots;
     
    +	// --------- resource manager --------
     	private TaskExecutorToResourceManagerConnection resourceManagerConnection;
     
     	// ------------------------------------------------------------------------
     
     	public TaskExecutor(
    +			TaskManagerConfiguration taskManagerConfig,
    +			ResourceID resourceID,
    +			InstanceConnectionInfo connectionInfo,
    +			MemoryManager memoryManager,
    +			IOManager ioManager,
    +			NetworkEnvironment networkEnvironment,
    +			int numberOfSlots,
     			RpcService rpcService,
    -			HighAvailabilityServices haServices,
    -			ResourceID resourceID) {
    +			HighAvailabilityServices haServices) {
     
     		super(rpcService);
     
    -		this.haServices = checkNotNull(haServices);
    +		this.taskManagerConfig = checkNotNull(taskManagerConfig);
     		this.resourceID = checkNotNull(resourceID);
    +		this.connectionInfo = checkNotNull(connectionInfo);
    +		this.memoryManager = checkNotNull(memoryManager);
    +		this.ioManager = checkNotNull(ioManager);
    +		this.networkEnvironment = checkNotNull(networkEnvironment);
    +		this.numberOfSlots = checkNotNull(numberOfSlots);
    +		this.haServices = checkNotNull(haServices);
    +	}
    +
    +	/**
    +	 * Starts and runs the TaskManager.
    +	 * <p/>
    +	 * This method first tries to select the network interface to use for the TaskManager
    +	 * communication. The network interface is used both for the actor communication
    +	 * (coordination) as well as for the data exchange between task managers. Unless
    +	 * the hostname/interface is explicitly configured in the configuration, this
    +	 * method will try out various interfaces and methods to connect to the JobManager
    +	 * and select the one where the connection attempt is successful.
    +	 * <p/>
    +	 * After selecting the network interface, this method brings up an actor system
    +	 * for the TaskManager and its actors, starts the TaskManager's services
    +	 * (library cache, shuffle network stack, ...), and starts the TaskManager itself.
    +	 *
    +	 * @param configuration    The configuration for the TaskManager.
    +	 * @param taskManagerClass The actor class to instantiate.
    +	 *                         Allows to use TaskManager subclasses for example for YARN.
    +	 */
    +	public static void selectNetworkInterfaceAndRunTaskManager(
    +		Configuration configuration,
    +		ResourceID resourceID,
    +		Class<? extends TaskManager> taskManagerClass) throws Exception {
    +
    +		Tuple2<String, Integer> tuple2 = selectNetworkInterfaceAndPort(configuration);
    --- End diff --
    
    Do we want to rely on Scala tuples here after the conversion from Scala to Java? I would suggest `InetAddress` here.


---
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.
---

[GitHub] flink pull request #2400: [FLINK-4363] Implement TaskManager basic startup o...

Posted by mxm <gi...@git.apache.org>.
Github user mxm commented on a diff in the pull request:

    https://github.com/apache/flink/pull/2400#discussion_r76274362
  
    --- Diff: flink-runtime/src/main/java/org/apache/flink/runtime/rpc/taskexecutor/TaskExecutor.java ---
    @@ -35,27 +79,634 @@
      */
     public class TaskExecutor extends RpcEndpoint<TaskExecutorGateway> {
     
    -	/** The unique resource ID of this TaskExecutor */
    +	private static final Logger LOG = LoggerFactory.getLogger(TaskExecutor.class);
    +
    +	/** Return code for critical errors during the runtime */
    +	private static final int RUNTIME_FAILURE_RETURN_CODE = 2;
    +
    +	/** The name of the TaskManager actor */
    +	private static final String TASK_MANAGER_NAME = "taskmanager";
    +
    +	/** The unique resource ID of this TaskManager */
     	private final ResourceID resourceID;
     
     	/** The access to the leader election and metadata storage services */
     	private final HighAvailabilityServices haServices;
     
    -	// --------- resource manager --------
    +	/** The task manager configuration */
    +	private final TaskManagerConfiguration taskManagerConfig;
    +
    +	/** The connection information of the task manager */
    +	private final InstanceConnectionInfo connectionInfo;
    +
    +	/** The I/O manager component in the task manager */
    +	private final IOManager ioManager;
    +
    +	/** The memory manager component in the task manager */
    +	private final MemoryManager memoryManager;
    +
    +	/** The network component in the task manager */
    +	private final NetworkEnvironment networkEnvironment;
    +
    +	/** The number of slots in the task manager, should be 1 for YARN */
    +	private final int numberOfSlots;
     
    +	// --------- resource manager --------
     	private TaskExecutorToResourceManagerConnection resourceManagerConnection;
     
     	// ------------------------------------------------------------------------
     
     	public TaskExecutor(
    +			TaskManagerConfiguration taskManagerConfig,
    +			ResourceID resourceID,
    +			InstanceConnectionInfo connectionInfo,
    +			MemoryManager memoryManager,
    +			IOManager ioManager,
    +			NetworkEnvironment networkEnvironment,
    +			int numberOfSlots,
     			RpcService rpcService,
    -			HighAvailabilityServices haServices,
    -			ResourceID resourceID) {
    +			HighAvailabilityServices haServices) {
     
     		super(rpcService);
     
    -		this.haServices = checkNotNull(haServices);
    +		this.taskManagerConfig = checkNotNull(taskManagerConfig);
     		this.resourceID = checkNotNull(resourceID);
    +		this.connectionInfo = checkNotNull(connectionInfo);
    +		this.memoryManager = checkNotNull(memoryManager);
    +		this.ioManager = checkNotNull(ioManager);
    +		this.networkEnvironment = checkNotNull(networkEnvironment);
    +		this.numberOfSlots = checkNotNull(numberOfSlots);
    +		this.haServices = checkNotNull(haServices);
    +	}
    +
    +	/**
    +	 * Starts and runs the TaskManager.
    +	 * <p/>
    +	 * This method first tries to select the network interface to use for the TaskManager
    +	 * communication. The network interface is used both for the actor communication
    +	 * (coordination) as well as for the data exchange between task managers. Unless
    +	 * the hostname/interface is explicitly configured in the configuration, this
    +	 * method will try out various interfaces and methods to connect to the JobManager
    +	 * and select the one where the connection attempt is successful.
    +	 * <p/>
    +	 * After selecting the network interface, this method brings up an actor system
    +	 * for the TaskManager and its actors, starts the TaskManager's services
    +	 * (library cache, shuffle network stack, ...), and starts the TaskManager itself.
    +	 *
    +	 * @param configuration    The configuration for the TaskManager.
    +	 * @param taskManagerClass The actor class to instantiate.
    +	 *                         Allows to use TaskManager subclasses for example for YARN.
    +	 */
    +	public static void selectNetworkInterfaceAndRunTaskManager(
    +		Configuration configuration,
    +		ResourceID resourceID,
    +		Class<? extends TaskManager> taskManagerClass) throws Exception {
    +
    +		Tuple2<String, Integer> tuple2 = selectNetworkInterfaceAndPort(configuration);
    +
    +		runTaskManager(tuple2._1(), resourceID, tuple2._2(), configuration, taskManagerClass);
    +	}
    +
    +	private static Tuple2<String, Integer> selectNetworkInterfaceAndPort(Configuration configuration)
    +		throws Exception {
    +		String taskManagerHostname = configuration.getString(ConfigConstants.TASK_MANAGER_HOSTNAME_KEY, null);
    +		if (taskManagerHostname != null) {
    +			LOG.info("Using configured hostname/address for TaskManager: " + taskManagerHostname);
    +		} else {
    +			LeaderRetrievalService leaderRetrievalService = LeaderRetrievalUtils.createLeaderRetrievalService(configuration);
    +			FiniteDuration lookupTimeout = AkkaUtils.getLookupTimeout(configuration);
    +
    +			InetAddress taskManagerAddress = LeaderRetrievalUtils.findConnectingAddress(leaderRetrievalService, lookupTimeout);
    +			taskManagerHostname = taskManagerAddress.getHostName();
    +			LOG.info("TaskManager will use hostname/address '{}' ({}) for communication.",
    +				taskManagerHostname, taskManagerAddress.getHostAddress());
    +		}
    +
    +		// if no task manager port has been configured, use 0 (system will pick any free port)
    +		int actorSystemPort = configuration.getInteger(ConfigConstants.TASK_MANAGER_IPC_PORT_KEY, 0);
    +		if (actorSystemPort < 0 || actorSystemPort > 65535) {
    +			throw new IllegalConfigurationException("Invalid value for '" +
    +				ConfigConstants.TASK_MANAGER_IPC_PORT_KEY +
    +				"' (port for the TaskManager actor system) : " + actorSystemPort +
    +				" - Leave config parameter empty or use 0 to let the system choose a port automatically.");
    +		}
    +
    +		return new Tuple2<>(taskManagerHostname, actorSystemPort);
    +	}
    +
    +	/**
    +	 * Starts and runs the TaskManager. Brings up an actor system for the TaskManager and its
    +	 * actors, starts the TaskManager's services (library cache, shuffle network stack, ...),
    +	 * and starts the TaskManager itself.
    +	 * <p/>
    +	 * This method will also spawn a process reaper for the TaskManager (kill the process if
    +	 * the actor fails) and optionally start the JVM memory logging thread.
    +	 *
    +	 * @param taskManagerHostname The hostname/address of the interface where the actor system
    +	 *                            will communicate.
    +	 * @param resourceID          The id of the resource which the task manager will run on.
    +	 * @param actorSystemPort     The port at which the actor system will communicate.
    +	 * @param configuration       The configuration for the TaskManager.
    +	 * @param taskManagerClass    The actor class to instantiate. Allows the use of TaskManager
    +	 *                            subclasses for example for YARN.
    +	 */
    +	private static void runTaskManager(
    +		String taskManagerHostname,
    +		ResourceID resourceID,
    +		int actorSystemPort,
    +		Configuration configuration,
    +		Class<? extends TaskManager> taskManagerClass) throws Exception {
    +
    +		LOG.info("Starting TaskManager");
    +
    +		// Bring up the TaskManager actor system first, bind it to the given address.
    +
    +		LOG.info("Starting TaskManager actor system at " +
    +			NetUtils.hostAndPortToUrlString(taskManagerHostname, actorSystemPort));
    +
    +		ActorSystem taskManagerSystem;
    +		try {
    +			Tuple2<String, Object> address = new Tuple2<String, Object>(taskManagerHostname, actorSystemPort);
    +			Config akkaConfig = AkkaUtils.getAkkaConfig(configuration, new Some<>(address));
    +			LOG.debug("Using akka configuration\n " + akkaConfig);
    +			taskManagerSystem = AkkaUtils.createActorSystem(akkaConfig);
    +		} catch (Throwable t) {
    +			if (t instanceof org.jboss.netty.channel.ChannelException) {
    +				Throwable cause = t.getCause();
    +				if (cause != null && t.getCause() instanceof java.net.BindException) {
    +					String address = NetUtils.hostAndPortToUrlString(taskManagerHostname, actorSystemPort);
    +					throw new IOException("Unable to bind TaskManager actor system to address " +
    +						address + " - " + cause.getMessage(), t);
    +				}
    +			}
    +			throw new Exception("Could not create TaskManager actor system", t);
    +		}
    +
    +		// start all the TaskManager services (network stack,  library cache, ...)
    +		// and the TaskManager actor
    +		try {
    +			LOG.info("Starting TaskManager actor");
    +			ActorRef taskManagerActor = startTaskManagerComponentsAndActor(
    +				configuration,
    +				resourceID,
    +				taskManagerSystem,
    +				taskManagerHostname,
    +				TASK_MANAGER_NAME,
    +				null,
    +				false,
    +				taskManagerClass);
    +
    +			// start a process reaper that watches the JobManager. If the TaskManager actor dies,
    +			// the process reaper will kill the JVM process (to ensure easy failure detection)
    +			LOG.debug("Starting TaskManager process reaper");
    +
    +			taskManagerSystem.actorOf(
    +				Props.create(ProcessReaper.class, taskManagerActor, LOG, RUNTIME_FAILURE_RETURN_CODE),
    +				"TaskManager_Process_Reaper");
    +
    +			// if desired, start the logging daemon that periodically logs the memory usage information
    +			if (LOG.isInfoEnabled() && configuration.getBoolean(
    +				ConfigConstants.TASK_MANAGER_DEBUG_MEMORY_USAGE_START_LOG_THREAD,
    +				ConfigConstants.DEFAULT_TASK_MANAGER_DEBUG_MEMORY_USAGE_START_LOG_THREAD)) {
    +				LOG.info("Starting periodic memory usage logger");
    +
    +				long interval = configuration.getLong(
    +					ConfigConstants.TASK_MANAGER_DEBUG_MEMORY_USAGE_LOG_INTERVAL_MS,
    +					ConfigConstants.DEFAULT_TASK_MANAGER_DEBUG_MEMORY_USAGE_LOG_INTERVAL_MS);
    +
    +				MemoryLogger logger = new MemoryLogger(LOG, interval, taskManagerSystem);
    +				logger.start();
    +			}
    +
    +			// block until everything is done
    +			taskManagerSystem.awaitTermination();
    +		} catch (Throwable t) {
    +			LOG.error("Error while starting up taskManager", t);
    +			try {
    +				taskManagerSystem.shutdown();
    +			} catch (Throwable tt) {
    +				LOG.warn("Could not cleanly shut down actor system", tt);
    +			}
    +			throw t;
    +		}
    +	}
    +
    +	// --------------------------------------------------------------------------
    +	//  Starting and running the TaskManager
    +	// --------------------------------------------------------------------------
    +
    +	/**
    +	 * @param configuration                 The configuration for the TaskManager.
    +	 * @param resourceID                    The id of the resource which the task manager will run on.
    +	 * @param actorSystem                  The actor system that should run the TaskManager actor.
    +	 * @param taskManagerHostname       The hostname/address that describes the TaskManager's data location.
    +	 * @param taskManagerActorName      Optionally the name of the TaskManager actor. If none is given,
    +	 *                                      the actor will use a random name.
    +	 * @param leaderRetrievalService        Optionally, a leader retrieval service can be provided. If none is given,
    +	 *                                      then a LeaderRetrievalService is constructed from the configuration.
    +	 * @param localTaskManagerCommunication If true, the TaskManager will not initiate the TCP network stack.
    +	 * @param taskManagerClass      The class of the TaskManager actor. May be used to give
    +	 *                                      subclasses that understand additional actor messages.
    +	 * @return An ActorRef to the TaskManager actor.
    +	 * @throws org.apache.flink.configuration.IllegalConfigurationException     Thrown, if the given config contains illegal values.
    +	 * @throws java.io.IOException      Thrown, if any of the I/O components (such as buffer pools,
    +	 *                                       I/O manager, ...) cannot be properly started.
    +	 * @throws java.lang.Exception      Thrown is some other error occurs while parsing the configuration
    +	 *                                      or starting the TaskManager components.
    +	 */
    +	public static ActorRef startTaskManagerComponentsAndActor(
    +		Configuration configuration,
    +		ResourceID resourceID,
    +		ActorSystem actorSystem,
    +		String taskManagerHostname,
    +		String taskManagerActorName,
    +		LeaderRetrievalService leaderRetrievalService,
    +		boolean localTaskManagerCommunication,
    +		Class<? extends TaskManager> taskManagerClass) throws Exception {
    +
    +		Tuple4<TaskManagerConfiguration, NetworkEnvironmentConfiguration, InstanceConnectionInfo, MemoryType> tuple4
    +			= parseTaskManagerConfiguration(configuration, taskManagerHostname, localTaskManagerCommunication);
    +
    +		TaskManagerConfiguration taskManagerConfig = tuple4._1();
    +		NetworkEnvironmentConfiguration netConfig = tuple4._2();
    +		InstanceConnectionInfo connectionInfo = tuple4._3();
    +		MemoryType memType = tuple4._4();
    +
    +		// pre-start checks
    +		checkTempDirs(taskManagerConfig.tmpDirPaths());
    +
    +		ExecutionContext executionContext = ExecutionContexts$.MODULE$.fromExecutor(new ForkJoinPool());
    +
    +		// we start the network first, to make sure it can allocate its buffers first
    +		NetworkEnvironment network = new NetworkEnvironment(executionContext, taskManagerConfig.timeout(), netConfig);
    +
    +		// computing the amount of memory to use depends on how much memory is available
    +		// it strictly needs to happen AFTER the network stack has been initialized
    +
    +		// check if a value has been configured
    +		long configuredMemory = configuration.getLong(ConfigConstants.TASK_MANAGER_MEMORY_SIZE_KEY, -1L);
    +		checkConfigParameter(configuredMemory == -1 || configuredMemory > 0, configuredMemory,
    +			ConfigConstants.TASK_MANAGER_MEMORY_SIZE_KEY,
    +			"MemoryManager needs at least one MB of memory. " +
    +				"If you leave this config parameter empty, the system automatically " +
    +				"pick a fraction of the available memory.");
    +
    +		long memorySize;
    --- End diff --
    
    This comment hasn't been addressed yet.


---
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.
---

[GitHub] flink pull request #2400: [FLINK-4363] Implement TaskManager basic startup o...

Posted by mxm <gi...@git.apache.org>.
Github user mxm commented on a diff in the pull request:

    https://github.com/apache/flink/pull/2400#discussion_r75645192
  
    --- Diff: flink-runtime/src/main/java/org/apache/flink/runtime/rpc/taskexecutor/TaskExecutor.java ---
    @@ -35,27 +79,634 @@
      */
     public class TaskExecutor extends RpcEndpoint<TaskExecutorGateway> {
     
    -	/** The unique resource ID of this TaskExecutor */
    +	private static final Logger LOG = LoggerFactory.getLogger(TaskExecutor.class);
    +
    +	/** Return code for critical errors during the runtime */
    +	private static final int RUNTIME_FAILURE_RETURN_CODE = 2;
    +
    +	/** The name of the TaskManager actor */
    +	private static final String TASK_MANAGER_NAME = "taskmanager";
    +
    +	/** The unique resource ID of this TaskManager */
     	private final ResourceID resourceID;
     
     	/** The access to the leader election and metadata storage services */
     	private final HighAvailabilityServices haServices;
     
    -	// --------- resource manager --------
    +	/** The task manager configuration */
    +	private final TaskManagerConfiguration taskManagerConfig;
    +
    +	/** The connection information of the task manager */
    +	private final InstanceConnectionInfo connectionInfo;
    +
    +	/** The I/O manager component in the task manager */
    +	private final IOManager ioManager;
    +
    +	/** The memory manager component in the task manager */
    +	private final MemoryManager memoryManager;
    +
    +	/** The network component in the task manager */
    +	private final NetworkEnvironment networkEnvironment;
    +
    +	/** The number of slots in the task manager, should be 1 for YARN */
    +	private final int numberOfSlots;
     
    +	// --------- resource manager --------
     	private TaskExecutorToResourceManagerConnection resourceManagerConnection;
     
     	// ------------------------------------------------------------------------
     
     	public TaskExecutor(
    +			TaskManagerConfiguration taskManagerConfig,
    +			ResourceID resourceID,
    +			InstanceConnectionInfo connectionInfo,
    +			MemoryManager memoryManager,
    +			IOManager ioManager,
    +			NetworkEnvironment networkEnvironment,
    +			int numberOfSlots,
     			RpcService rpcService,
    -			HighAvailabilityServices haServices,
    -			ResourceID resourceID) {
    +			HighAvailabilityServices haServices) {
     
     		super(rpcService);
     
    -		this.haServices = checkNotNull(haServices);
    +		this.taskManagerConfig = checkNotNull(taskManagerConfig);
     		this.resourceID = checkNotNull(resourceID);
    +		this.connectionInfo = checkNotNull(connectionInfo);
    +		this.memoryManager = checkNotNull(memoryManager);
    +		this.ioManager = checkNotNull(ioManager);
    +		this.networkEnvironment = checkNotNull(networkEnvironment);
    +		this.numberOfSlots = checkNotNull(numberOfSlots);
    +		this.haServices = checkNotNull(haServices);
    +	}
    +
    +	/**
    +	 * Starts and runs the TaskManager.
    +	 * <p/>
    +	 * This method first tries to select the network interface to use for the TaskManager
    +	 * communication. The network interface is used both for the actor communication
    +	 * (coordination) as well as for the data exchange between task managers. Unless
    +	 * the hostname/interface is explicitly configured in the configuration, this
    +	 * method will try out various interfaces and methods to connect to the JobManager
    +	 * and select the one where the connection attempt is successful.
    +	 * <p/>
    +	 * After selecting the network interface, this method brings up an actor system
    +	 * for the TaskManager and its actors, starts the TaskManager's services
    +	 * (library cache, shuffle network stack, ...), and starts the TaskManager itself.
    +	 *
    +	 * @param configuration    The configuration for the TaskManager.
    +	 * @param taskManagerClass The actor class to instantiate.
    +	 *                         Allows to use TaskManager subclasses for example for YARN.
    +	 */
    +	public static void selectNetworkInterfaceAndRunTaskManager(
    +		Configuration configuration,
    +		ResourceID resourceID,
    +		Class<? extends TaskManager> taskManagerClass) throws Exception {
    +
    +		Tuple2<String, Integer> tuple2 = selectNetworkInterfaceAndPort(configuration);
    +
    +		runTaskManager(tuple2._1(), resourceID, tuple2._2(), configuration, taskManagerClass);
    +	}
    +
    +	private static Tuple2<String, Integer> selectNetworkInterfaceAndPort(Configuration configuration)
    +		throws Exception {
    +		String taskManagerHostname = configuration.getString(ConfigConstants.TASK_MANAGER_HOSTNAME_KEY, null);
    +		if (taskManagerHostname != null) {
    +			LOG.info("Using configured hostname/address for TaskManager: " + taskManagerHostname);
    +		} else {
    +			LeaderRetrievalService leaderRetrievalService = LeaderRetrievalUtils.createLeaderRetrievalService(configuration);
    +			FiniteDuration lookupTimeout = AkkaUtils.getLookupTimeout(configuration);
    +
    +			InetAddress taskManagerAddress = LeaderRetrievalUtils.findConnectingAddress(leaderRetrievalService, lookupTimeout);
    +			taskManagerHostname = taskManagerAddress.getHostName();
    +			LOG.info("TaskManager will use hostname/address '{}' ({}) for communication.",
    +				taskManagerHostname, taskManagerAddress.getHostAddress());
    +		}
    +
    +		// if no task manager port has been configured, use 0 (system will pick any free port)
    +		int actorSystemPort = configuration.getInteger(ConfigConstants.TASK_MANAGER_IPC_PORT_KEY, 0);
    +		if (actorSystemPort < 0 || actorSystemPort > 65535) {
    +			throw new IllegalConfigurationException("Invalid value for '" +
    +				ConfigConstants.TASK_MANAGER_IPC_PORT_KEY +
    +				"' (port for the TaskManager actor system) : " + actorSystemPort +
    +				" - Leave config parameter empty or use 0 to let the system choose a port automatically.");
    +		}
    +
    +		return new Tuple2<>(taskManagerHostname, actorSystemPort);
    +	}
    +
    +	/**
    +	 * Starts and runs the TaskManager. Brings up an actor system for the TaskManager and its
    +	 * actors, starts the TaskManager's services (library cache, shuffle network stack, ...),
    +	 * and starts the TaskManager itself.
    +	 * <p/>
    +	 * This method will also spawn a process reaper for the TaskManager (kill the process if
    +	 * the actor fails) and optionally start the JVM memory logging thread.
    +	 *
    +	 * @param taskManagerHostname The hostname/address of the interface where the actor system
    +	 *                            will communicate.
    +	 * @param resourceID          The id of the resource which the task manager will run on.
    +	 * @param actorSystemPort     The port at which the actor system will communicate.
    +	 * @param configuration       The configuration for the TaskManager.
    +	 * @param taskManagerClass    The actor class to instantiate. Allows the use of TaskManager
    +	 *                            subclasses for example for YARN.
    +	 */
    +	private static void runTaskManager(
    +		String taskManagerHostname,
    +		ResourceID resourceID,
    +		int actorSystemPort,
    +		Configuration configuration,
    +		Class<? extends TaskManager> taskManagerClass) throws Exception {
    +
    +		LOG.info("Starting TaskManager");
    +
    +		// Bring up the TaskManager actor system first, bind it to the given address.
    +
    +		LOG.info("Starting TaskManager actor system at " +
    +			NetUtils.hostAndPortToUrlString(taskManagerHostname, actorSystemPort));
    +
    +		ActorSystem taskManagerSystem;
    +		try {
    +			Tuple2<String, Object> address = new Tuple2<String, Object>(taskManagerHostname, actorSystemPort);
    +			Config akkaConfig = AkkaUtils.getAkkaConfig(configuration, new Some<>(address));
    +			LOG.debug("Using akka configuration\n " + akkaConfig);
    +			taskManagerSystem = AkkaUtils.createActorSystem(akkaConfig);
    +		} catch (Throwable t) {
    +			if (t instanceof org.jboss.netty.channel.ChannelException) {
    +				Throwable cause = t.getCause();
    +				if (cause != null && t.getCause() instanceof java.net.BindException) {
    +					String address = NetUtils.hostAndPortToUrlString(taskManagerHostname, actorSystemPort);
    +					throw new IOException("Unable to bind TaskManager actor system to address " +
    +						address + " - " + cause.getMessage(), t);
    +				}
    +			}
    +			throw new Exception("Could not create TaskManager actor system", t);
    +		}
    +
    +		// start all the TaskManager services (network stack,  library cache, ...)
    +		// and the TaskManager actor
    +		try {
    +			LOG.info("Starting TaskManager actor");
    +			ActorRef taskManagerActor = startTaskManagerComponentsAndActor(
    +				configuration,
    +				resourceID,
    +				taskManagerSystem,
    +				taskManagerHostname,
    +				TASK_MANAGER_NAME,
    +				null,
    +				false,
    +				taskManagerClass);
    +
    +			// start a process reaper that watches the JobManager. If the TaskManager actor dies,
    +			// the process reaper will kill the JVM process (to ensure easy failure detection)
    +			LOG.debug("Starting TaskManager process reaper");
    +
    +			taskManagerSystem.actorOf(
    +				Props.create(ProcessReaper.class, taskManagerActor, LOG, RUNTIME_FAILURE_RETURN_CODE),
    +				"TaskManager_Process_Reaper");
    +
    +			// if desired, start the logging daemon that periodically logs the memory usage information
    +			if (LOG.isInfoEnabled() && configuration.getBoolean(
    +				ConfigConstants.TASK_MANAGER_DEBUG_MEMORY_USAGE_START_LOG_THREAD,
    +				ConfigConstants.DEFAULT_TASK_MANAGER_DEBUG_MEMORY_USAGE_START_LOG_THREAD)) {
    +				LOG.info("Starting periodic memory usage logger");
    +
    +				long interval = configuration.getLong(
    +					ConfigConstants.TASK_MANAGER_DEBUG_MEMORY_USAGE_LOG_INTERVAL_MS,
    +					ConfigConstants.DEFAULT_TASK_MANAGER_DEBUG_MEMORY_USAGE_LOG_INTERVAL_MS);
    +
    +				MemoryLogger logger = new MemoryLogger(LOG, interval, taskManagerSystem);
    +				logger.start();
    +			}
    +
    +			// block until everything is done
    +			taskManagerSystem.awaitTermination();
    +		} catch (Throwable t) {
    +			LOG.error("Error while starting up taskManager", t);
    +			try {
    +				taskManagerSystem.shutdown();
    +			} catch (Throwable tt) {
    +				LOG.warn("Could not cleanly shut down actor system", tt);
    +			}
    +			throw t;
    +		}
    +	}
    +
    +	// --------------------------------------------------------------------------
    +	//  Starting and running the TaskManager
    +	// --------------------------------------------------------------------------
    +
    +	/**
    +	 * @param configuration                 The configuration for the TaskManager.
    +	 * @param resourceID                    The id of the resource which the task manager will run on.
    +	 * @param actorSystem                  The actor system that should run the TaskManager actor.
    +	 * @param taskManagerHostname       The hostname/address that describes the TaskManager's data location.
    +	 * @param taskManagerActorName      Optionally the name of the TaskManager actor. If none is given,
    +	 *                                      the actor will use a random name.
    +	 * @param leaderRetrievalService        Optionally, a leader retrieval service can be provided. If none is given,
    +	 *                                      then a LeaderRetrievalService is constructed from the configuration.
    +	 * @param localTaskManagerCommunication If true, the TaskManager will not initiate the TCP network stack.
    +	 * @param taskManagerClass      The class of the TaskManager actor. May be used to give
    +	 *                                      subclasses that understand additional actor messages.
    +	 * @return An ActorRef to the TaskManager actor.
    +	 * @throws org.apache.flink.configuration.IllegalConfigurationException     Thrown, if the given config contains illegal values.
    +	 * @throws java.io.IOException      Thrown, if any of the I/O components (such as buffer pools,
    +	 *                                       I/O manager, ...) cannot be properly started.
    +	 * @throws java.lang.Exception      Thrown is some other error occurs while parsing the configuration
    +	 *                                      or starting the TaskManager components.
    +	 */
    +	public static ActorRef startTaskManagerComponentsAndActor(
    +		Configuration configuration,
    +		ResourceID resourceID,
    +		ActorSystem actorSystem,
    +		String taskManagerHostname,
    +		String taskManagerActorName,
    +		LeaderRetrievalService leaderRetrievalService,
    +		boolean localTaskManagerCommunication,
    +		Class<? extends TaskManager> taskManagerClass) throws Exception {
    +
    +		Tuple4<TaskManagerConfiguration, NetworkEnvironmentConfiguration, InstanceConnectionInfo, MemoryType> tuple4
    +			= parseTaskManagerConfiguration(configuration, taskManagerHostname, localTaskManagerCommunication);
    +
    +		TaskManagerConfiguration taskManagerConfig = tuple4._1();
    +		NetworkEnvironmentConfiguration netConfig = tuple4._2();
    +		InstanceConnectionInfo connectionInfo = tuple4._3();
    +		MemoryType memType = tuple4._4();
    +
    +		// pre-start checks
    +		checkTempDirs(taskManagerConfig.tmpDirPaths());
    +
    +		ExecutionContext executionContext = ExecutionContexts$.MODULE$.fromExecutor(new ForkJoinPool());
    +
    +		// we start the network first, to make sure it can allocate its buffers first
    +		NetworkEnvironment network = new NetworkEnvironment(executionContext, taskManagerConfig.timeout(), netConfig);
    +
    +		// computing the amount of memory to use depends on how much memory is available
    +		// it strictly needs to happen AFTER the network stack has been initialized
    +
    +		// check if a value has been configured
    +		long configuredMemory = configuration.getLong(ConfigConstants.TASK_MANAGER_MEMORY_SIZE_KEY, -1L);
    +		checkConfigParameter(configuredMemory == -1 || configuredMemory > 0, configuredMemory,
    +			ConfigConstants.TASK_MANAGER_MEMORY_SIZE_KEY,
    +			"MemoryManager needs at least one MB of memory. " +
    +				"If you leave this config parameter empty, the system automatically " +
    +				"pick a fraction of the available memory.");
    +
    +		long memorySize;
    +		boolean preAllocateMemory = configuration.getBoolean(
    +			ConfigConstants.TASK_MANAGER_MEMORY_PRE_ALLOCATE_KEY,
    +			ConfigConstants.DEFAULT_TASK_MANAGER_MEMORY_PRE_ALLOCATE);
    +		if (configuredMemory > 0) {
    +			if (preAllocateMemory) {
    +				LOG.info("Using {} MB for managed memory." , configuredMemory);
    +			} else {
    +				LOG.info("Limiting managed memory to {} MB, memory will be allocated lazily." , configuredMemory);
    +			}
    +			memorySize = configuredMemory << 20; // megabytes to bytes
    +		} else {
    +			float fraction = configuration.getFloat(
    +				ConfigConstants.TASK_MANAGER_MEMORY_FRACTION_KEY,
    +				ConfigConstants.DEFAULT_MEMORY_MANAGER_MEMORY_FRACTION);
    +			checkConfigParameter(fraction > 0.0f && fraction < 1.0f, fraction,
    +				ConfigConstants.TASK_MANAGER_MEMORY_FRACTION_KEY,
    +				"MemoryManager fraction of the free memory must be between 0.0 and 1.0");
    +
    +			if (memType == MemoryType.HEAP) {
    +				long relativeMemSize = (long) (EnvironmentInformation.getSizeOfFreeHeapMemoryWithDefrag() * fraction);
    +				if (preAllocateMemory) {
    +					LOG.info("Using {} of the currently free heap space for managed heap memory ({} MB)." ,
    +						fraction , relativeMemSize >> 20);
    +				} else {
    +					LOG.info("Limiting managed memory to {} of the currently free heap space ({} MB), " +
    +						"memory will be allocated lazily." , fraction , relativeMemSize >> 20);
    +				}
    +				memorySize = relativeMemSize;
    +			} else if (memType == MemoryType.OFF_HEAP) {
    +				// The maximum heap memory has been adjusted according to the fraction
    +				long maxMemory = EnvironmentInformation.getMaxJvmHeapMemory();
    +				long directMemorySize = (long) (maxMemory / (1.0 - fraction) * fraction);
    +				if (preAllocateMemory) {
    +					LOG.info("Using {} of the maximum memory size for managed off-heap memory ({} MB)." ,
    +						fraction, directMemorySize >> 20);
    +				} else {
    +					LOG.info("Limiting managed memory to {} of the maximum memory size ({} MB)," +
    +						" memory will be allocated lazily.", fraction, directMemorySize >> 20);
    +				}
    +				memorySize = directMemorySize;
    +			} else {
    +				throw new RuntimeException("No supported memory type detected.");
    +			}
    +		}
    +
    +		// now start the memory manager
    +		MemoryManager memoryManager;
    +		try {
    +			memoryManager = new MemoryManager(
    +				memorySize,
    +				taskManagerConfig.numberOfSlots(),
    +				netConfig.networkBufferSize(),
    +				memType,
    +				preAllocateMemory);
    +		} catch (OutOfMemoryError e) {
    +			if (memType == MemoryType.HEAP) {
    +				throw new Exception("OutOfMemory error (" + e.getMessage() +
    +					") while allocating the TaskManager heap memory (" + memorySize + " bytes).", e);
    +			} else if (memType == MemoryType.OFF_HEAP) {
    +				throw new Exception("OutOfMemory error (" + e.getMessage() +
    +					") while allocating the TaskManager off-heap memory (" + memorySize +
    +					" bytes).Try increasing the maximum direct memory (-XX:MaxDirectMemorySize)", e);
    +			} else {
    +				throw e;
    +			}
    +		}
    +
    +		// start the I/O manager, it will create some temp directories.
    +		IOManager ioManager = new IOManagerAsync(taskManagerConfig.tmpDirPaths());
    +
    +		if (leaderRetrievalService == null){
    +			leaderRetrievalService = LeaderRetrievalUtils.createLeaderRetrievalService(configuration);
    --- End diff --
    
    I think it would be better to enforce a non-null implementation in the constructor.


---
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.
---

[GitHub] flink pull request #2400: [FLINK-4363] Implement TaskManager basic startup o...

Posted by mxm <gi...@git.apache.org>.
Github user mxm commented on a diff in the pull request:

    https://github.com/apache/flink/pull/2400#discussion_r75644987
  
    --- Diff: flink-runtime/src/main/java/org/apache/flink/runtime/rpc/taskexecutor/TaskExecutor.java ---
    @@ -35,27 +79,634 @@
      */
     public class TaskExecutor extends RpcEndpoint<TaskExecutorGateway> {
     
    -	/** The unique resource ID of this TaskExecutor */
    +	private static final Logger LOG = LoggerFactory.getLogger(TaskExecutor.class);
    +
    +	/** Return code for critical errors during the runtime */
    +	private static final int RUNTIME_FAILURE_RETURN_CODE = 2;
    +
    +	/** The name of the TaskManager actor */
    +	private static final String TASK_MANAGER_NAME = "taskmanager";
    +
    +	/** The unique resource ID of this TaskManager */
     	private final ResourceID resourceID;
     
     	/** The access to the leader election and metadata storage services */
     	private final HighAvailabilityServices haServices;
     
    -	// --------- resource manager --------
    +	/** The task manager configuration */
    +	private final TaskManagerConfiguration taskManagerConfig;
    +
    +	/** The connection information of the task manager */
    +	private final InstanceConnectionInfo connectionInfo;
    +
    +	/** The I/O manager component in the task manager */
    +	private final IOManager ioManager;
    +
    +	/** The memory manager component in the task manager */
    +	private final MemoryManager memoryManager;
    +
    +	/** The network component in the task manager */
    +	private final NetworkEnvironment networkEnvironment;
    +
    +	/** The number of slots in the task manager, should be 1 for YARN */
    +	private final int numberOfSlots;
     
    +	// --------- resource manager --------
     	private TaskExecutorToResourceManagerConnection resourceManagerConnection;
     
     	// ------------------------------------------------------------------------
     
     	public TaskExecutor(
    +			TaskManagerConfiguration taskManagerConfig,
    +			ResourceID resourceID,
    +			InstanceConnectionInfo connectionInfo,
    +			MemoryManager memoryManager,
    +			IOManager ioManager,
    +			NetworkEnvironment networkEnvironment,
    +			int numberOfSlots,
     			RpcService rpcService,
    -			HighAvailabilityServices haServices,
    -			ResourceID resourceID) {
    +			HighAvailabilityServices haServices) {
     
     		super(rpcService);
     
    -		this.haServices = checkNotNull(haServices);
    +		this.taskManagerConfig = checkNotNull(taskManagerConfig);
     		this.resourceID = checkNotNull(resourceID);
    +		this.connectionInfo = checkNotNull(connectionInfo);
    +		this.memoryManager = checkNotNull(memoryManager);
    +		this.ioManager = checkNotNull(ioManager);
    +		this.networkEnvironment = checkNotNull(networkEnvironment);
    +		this.numberOfSlots = checkNotNull(numberOfSlots);
    +		this.haServices = checkNotNull(haServices);
    +	}
    +
    +	/**
    +	 * Starts and runs the TaskManager.
    +	 * <p/>
    +	 * This method first tries to select the network interface to use for the TaskManager
    +	 * communication. The network interface is used both for the actor communication
    +	 * (coordination) as well as for the data exchange between task managers. Unless
    +	 * the hostname/interface is explicitly configured in the configuration, this
    +	 * method will try out various interfaces and methods to connect to the JobManager
    +	 * and select the one where the connection attempt is successful.
    +	 * <p/>
    +	 * After selecting the network interface, this method brings up an actor system
    +	 * for the TaskManager and its actors, starts the TaskManager's services
    +	 * (library cache, shuffle network stack, ...), and starts the TaskManager itself.
    +	 *
    +	 * @param configuration    The configuration for the TaskManager.
    +	 * @param taskManagerClass The actor class to instantiate.
    +	 *                         Allows to use TaskManager subclasses for example for YARN.
    +	 */
    +	public static void selectNetworkInterfaceAndRunTaskManager(
    +		Configuration configuration,
    +		ResourceID resourceID,
    +		Class<? extends TaskManager> taskManagerClass) throws Exception {
    +
    +		Tuple2<String, Integer> tuple2 = selectNetworkInterfaceAndPort(configuration);
    +
    +		runTaskManager(tuple2._1(), resourceID, tuple2._2(), configuration, taskManagerClass);
    +	}
    +
    +	private static Tuple2<String, Integer> selectNetworkInterfaceAndPort(Configuration configuration)
    +		throws Exception {
    +		String taskManagerHostname = configuration.getString(ConfigConstants.TASK_MANAGER_HOSTNAME_KEY, null);
    +		if (taskManagerHostname != null) {
    +			LOG.info("Using configured hostname/address for TaskManager: " + taskManagerHostname);
    +		} else {
    +			LeaderRetrievalService leaderRetrievalService = LeaderRetrievalUtils.createLeaderRetrievalService(configuration);
    +			FiniteDuration lookupTimeout = AkkaUtils.getLookupTimeout(configuration);
    +
    +			InetAddress taskManagerAddress = LeaderRetrievalUtils.findConnectingAddress(leaderRetrievalService, lookupTimeout);
    +			taskManagerHostname = taskManagerAddress.getHostName();
    +			LOG.info("TaskManager will use hostname/address '{}' ({}) for communication.",
    +				taskManagerHostname, taskManagerAddress.getHostAddress());
    +		}
    +
    +		// if no task manager port has been configured, use 0 (system will pick any free port)
    +		int actorSystemPort = configuration.getInteger(ConfigConstants.TASK_MANAGER_IPC_PORT_KEY, 0);
    +		if (actorSystemPort < 0 || actorSystemPort > 65535) {
    +			throw new IllegalConfigurationException("Invalid value for '" +
    +				ConfigConstants.TASK_MANAGER_IPC_PORT_KEY +
    +				"' (port for the TaskManager actor system) : " + actorSystemPort +
    +				" - Leave config parameter empty or use 0 to let the system choose a port automatically.");
    +		}
    +
    +		return new Tuple2<>(taskManagerHostname, actorSystemPort);
    +	}
    +
    +	/**
    +	 * Starts and runs the TaskManager. Brings up an actor system for the TaskManager and its
    +	 * actors, starts the TaskManager's services (library cache, shuffle network stack, ...),
    +	 * and starts the TaskManager itself.
    +	 * <p/>
    +	 * This method will also spawn a process reaper for the TaskManager (kill the process if
    +	 * the actor fails) and optionally start the JVM memory logging thread.
    +	 *
    +	 * @param taskManagerHostname The hostname/address of the interface where the actor system
    +	 *                            will communicate.
    +	 * @param resourceID          The id of the resource which the task manager will run on.
    +	 * @param actorSystemPort     The port at which the actor system will communicate.
    +	 * @param configuration       The configuration for the TaskManager.
    +	 * @param taskManagerClass    The actor class to instantiate. Allows the use of TaskManager
    +	 *                            subclasses for example for YARN.
    +	 */
    +	private static void runTaskManager(
    +		String taskManagerHostname,
    +		ResourceID resourceID,
    +		int actorSystemPort,
    +		Configuration configuration,
    +		Class<? extends TaskManager> taskManagerClass) throws Exception {
    +
    +		LOG.info("Starting TaskManager");
    +
    +		// Bring up the TaskManager actor system first, bind it to the given address.
    +
    +		LOG.info("Starting TaskManager actor system at " +
    +			NetUtils.hostAndPortToUrlString(taskManagerHostname, actorSystemPort));
    +
    +		ActorSystem taskManagerSystem;
    +		try {
    +			Tuple2<String, Object> address = new Tuple2<String, Object>(taskManagerHostname, actorSystemPort);
    +			Config akkaConfig = AkkaUtils.getAkkaConfig(configuration, new Some<>(address));
    +			LOG.debug("Using akka configuration\n " + akkaConfig);
    +			taskManagerSystem = AkkaUtils.createActorSystem(akkaConfig);
    +		} catch (Throwable t) {
    +			if (t instanceof org.jboss.netty.channel.ChannelException) {
    +				Throwable cause = t.getCause();
    +				if (cause != null && t.getCause() instanceof java.net.BindException) {
    +					String address = NetUtils.hostAndPortToUrlString(taskManagerHostname, actorSystemPort);
    +					throw new IOException("Unable to bind TaskManager actor system to address " +
    +						address + " - " + cause.getMessage(), t);
    +				}
    +			}
    +			throw new Exception("Could not create TaskManager actor system", t);
    +		}
    +
    +		// start all the TaskManager services (network stack,  library cache, ...)
    +		// and the TaskManager actor
    +		try {
    +			LOG.info("Starting TaskManager actor");
    +			ActorRef taskManagerActor = startTaskManagerComponentsAndActor(
    +				configuration,
    +				resourceID,
    +				taskManagerSystem,
    +				taskManagerHostname,
    +				TASK_MANAGER_NAME,
    +				null,
    +				false,
    +				taskManagerClass);
    +
    +			// start a process reaper that watches the JobManager. If the TaskManager actor dies,
    +			// the process reaper will kill the JVM process (to ensure easy failure detection)
    +			LOG.debug("Starting TaskManager process reaper");
    +
    +			taskManagerSystem.actorOf(
    +				Props.create(ProcessReaper.class, taskManagerActor, LOG, RUNTIME_FAILURE_RETURN_CODE),
    +				"TaskManager_Process_Reaper");
    +
    +			// if desired, start the logging daemon that periodically logs the memory usage information
    +			if (LOG.isInfoEnabled() && configuration.getBoolean(
    +				ConfigConstants.TASK_MANAGER_DEBUG_MEMORY_USAGE_START_LOG_THREAD,
    +				ConfigConstants.DEFAULT_TASK_MANAGER_DEBUG_MEMORY_USAGE_START_LOG_THREAD)) {
    +				LOG.info("Starting periodic memory usage logger");
    +
    +				long interval = configuration.getLong(
    +					ConfigConstants.TASK_MANAGER_DEBUG_MEMORY_USAGE_LOG_INTERVAL_MS,
    +					ConfigConstants.DEFAULT_TASK_MANAGER_DEBUG_MEMORY_USAGE_LOG_INTERVAL_MS);
    +
    +				MemoryLogger logger = new MemoryLogger(LOG, interval, taskManagerSystem);
    +				logger.start();
    +			}
    +
    +			// block until everything is done
    +			taskManagerSystem.awaitTermination();
    +		} catch (Throwable t) {
    +			LOG.error("Error while starting up taskManager", t);
    +			try {
    +				taskManagerSystem.shutdown();
    +			} catch (Throwable tt) {
    +				LOG.warn("Could not cleanly shut down actor system", tt);
    +			}
    +			throw t;
    +		}
    +	}
    +
    +	// --------------------------------------------------------------------------
    +	//  Starting and running the TaskManager
    +	// --------------------------------------------------------------------------
    +
    +	/**
    +	 * @param configuration                 The configuration for the TaskManager.
    +	 * @param resourceID                    The id of the resource which the task manager will run on.
    +	 * @param actorSystem                  The actor system that should run the TaskManager actor.
    +	 * @param taskManagerHostname       The hostname/address that describes the TaskManager's data location.
    +	 * @param taskManagerActorName      Optionally the name of the TaskManager actor. If none is given,
    +	 *                                      the actor will use a random name.
    +	 * @param leaderRetrievalService        Optionally, a leader retrieval service can be provided. If none is given,
    +	 *                                      then a LeaderRetrievalService is constructed from the configuration.
    +	 * @param localTaskManagerCommunication If true, the TaskManager will not initiate the TCP network stack.
    +	 * @param taskManagerClass      The class of the TaskManager actor. May be used to give
    +	 *                                      subclasses that understand additional actor messages.
    +	 * @return An ActorRef to the TaskManager actor.
    +	 * @throws org.apache.flink.configuration.IllegalConfigurationException     Thrown, if the given config contains illegal values.
    +	 * @throws java.io.IOException      Thrown, if any of the I/O components (such as buffer pools,
    +	 *                                       I/O manager, ...) cannot be properly started.
    +	 * @throws java.lang.Exception      Thrown is some other error occurs while parsing the configuration
    +	 *                                      or starting the TaskManager components.
    +	 */
    +	public static ActorRef startTaskManagerComponentsAndActor(
    +		Configuration configuration,
    +		ResourceID resourceID,
    +		ActorSystem actorSystem,
    +		String taskManagerHostname,
    +		String taskManagerActorName,
    +		LeaderRetrievalService leaderRetrievalService,
    +		boolean localTaskManagerCommunication,
    +		Class<? extends TaskManager> taskManagerClass) throws Exception {
    +
    +		Tuple4<TaskManagerConfiguration, NetworkEnvironmentConfiguration, InstanceConnectionInfo, MemoryType> tuple4
    +			= parseTaskManagerConfiguration(configuration, taskManagerHostname, localTaskManagerCommunication);
    +
    +		TaskManagerConfiguration taskManagerConfig = tuple4._1();
    +		NetworkEnvironmentConfiguration netConfig = tuple4._2();
    +		InstanceConnectionInfo connectionInfo = tuple4._3();
    +		MemoryType memType = tuple4._4();
    +
    +		// pre-start checks
    +		checkTempDirs(taskManagerConfig.tmpDirPaths());
    +
    +		ExecutionContext executionContext = ExecutionContexts$.MODULE$.fromExecutor(new ForkJoinPool());
    +
    +		// we start the network first, to make sure it can allocate its buffers first
    +		NetworkEnvironment network = new NetworkEnvironment(executionContext, taskManagerConfig.timeout(), netConfig);
    +
    +		// computing the amount of memory to use depends on how much memory is available
    +		// it strictly needs to happen AFTER the network stack has been initialized
    +
    +		// check if a value has been configured
    +		long configuredMemory = configuration.getLong(ConfigConstants.TASK_MANAGER_MEMORY_SIZE_KEY, -1L);
    +		checkConfigParameter(configuredMemory == -1 || configuredMemory > 0, configuredMemory,
    +			ConfigConstants.TASK_MANAGER_MEMORY_SIZE_KEY,
    +			"MemoryManager needs at least one MB of memory. " +
    +				"If you leave this config parameter empty, the system automatically " +
    +				"pick a fraction of the available memory.");
    +
    +		long memorySize;
    +		boolean preAllocateMemory = configuration.getBoolean(
    +			ConfigConstants.TASK_MANAGER_MEMORY_PRE_ALLOCATE_KEY,
    +			ConfigConstants.DEFAULT_TASK_MANAGER_MEMORY_PRE_ALLOCATE);
    +		if (configuredMemory > 0) {
    +			if (preAllocateMemory) {
    +				LOG.info("Using {} MB for managed memory." , configuredMemory);
    +			} else {
    +				LOG.info("Limiting managed memory to {} MB, memory will be allocated lazily." , configuredMemory);
    +			}
    +			memorySize = configuredMemory << 20; // megabytes to bytes
    +		} else {
    +			float fraction = configuration.getFloat(
    +				ConfigConstants.TASK_MANAGER_MEMORY_FRACTION_KEY,
    +				ConfigConstants.DEFAULT_MEMORY_MANAGER_MEMORY_FRACTION);
    +			checkConfigParameter(fraction > 0.0f && fraction < 1.0f, fraction,
    +				ConfigConstants.TASK_MANAGER_MEMORY_FRACTION_KEY,
    +				"MemoryManager fraction of the free memory must be between 0.0 and 1.0");
    +
    +			if (memType == MemoryType.HEAP) {
    +				long relativeMemSize = (long) (EnvironmentInformation.getSizeOfFreeHeapMemoryWithDefrag() * fraction);
    +				if (preAllocateMemory) {
    +					LOG.info("Using {} of the currently free heap space for managed heap memory ({} MB)." ,
    +						fraction , relativeMemSize >> 20);
    +				} else {
    +					LOG.info("Limiting managed memory to {} of the currently free heap space ({} MB), " +
    +						"memory will be allocated lazily." , fraction , relativeMemSize >> 20);
    +				}
    +				memorySize = relativeMemSize;
    +			} else if (memType == MemoryType.OFF_HEAP) {
    +				// The maximum heap memory has been adjusted according to the fraction
    +				long maxMemory = EnvironmentInformation.getMaxJvmHeapMemory();
    +				long directMemorySize = (long) (maxMemory / (1.0 - fraction) * fraction);
    +				if (preAllocateMemory) {
    +					LOG.info("Using {} of the maximum memory size for managed off-heap memory ({} MB)." ,
    +						fraction, directMemorySize >> 20);
    +				} else {
    +					LOG.info("Limiting managed memory to {} of the maximum memory size ({} MB)," +
    +						" memory will be allocated lazily.", fraction, directMemorySize >> 20);
    +				}
    +				memorySize = directMemorySize;
    +			} else {
    +				throw new RuntimeException("No supported memory type detected.");
    +			}
    +		}
    +
    +		// now start the memory manager
    +		MemoryManager memoryManager;
    --- End diff --
    
    Could be declared `final` to make assignment explicit.


---
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.
---

[GitHub] flink pull request #2400: [FLINK-4363] Implement TaskManager basic startup o...

Posted by mxm <gi...@git.apache.org>.
Github user mxm commented on a diff in the pull request:

    https://github.com/apache/flink/pull/2400#discussion_r75647510
  
    --- Diff: flink-runtime/src/main/java/org/apache/flink/runtime/rpc/taskexecutor/TaskExecutor.java ---
    @@ -35,27 +79,634 @@
      */
     public class TaskExecutor extends RpcEndpoint<TaskExecutorGateway> {
     
    -	/** The unique resource ID of this TaskExecutor */
    +	private static final Logger LOG = LoggerFactory.getLogger(TaskExecutor.class);
    +
    +	/** Return code for critical errors during the runtime */
    +	private static final int RUNTIME_FAILURE_RETURN_CODE = 2;
    +
    +	/** The name of the TaskManager actor */
    +	private static final String TASK_MANAGER_NAME = "taskmanager";
    +
    +	/** The unique resource ID of this TaskManager */
     	private final ResourceID resourceID;
     
     	/** The access to the leader election and metadata storage services */
     	private final HighAvailabilityServices haServices;
     
    -	// --------- resource manager --------
    +	/** The task manager configuration */
    +	private final TaskManagerConfiguration taskManagerConfig;
    +
    +	/** The connection information of the task manager */
    +	private final InstanceConnectionInfo connectionInfo;
    +
    +	/** The I/O manager component in the task manager */
    +	private final IOManager ioManager;
    +
    +	/** The memory manager component in the task manager */
    +	private final MemoryManager memoryManager;
    +
    +	/** The network component in the task manager */
    +	private final NetworkEnvironment networkEnvironment;
    +
    +	/** The number of slots in the task manager, should be 1 for YARN */
    +	private final int numberOfSlots;
     
    +	// --------- resource manager --------
     	private TaskExecutorToResourceManagerConnection resourceManagerConnection;
     
     	// ------------------------------------------------------------------------
     
     	public TaskExecutor(
    +			TaskManagerConfiguration taskManagerConfig,
    +			ResourceID resourceID,
    +			InstanceConnectionInfo connectionInfo,
    +			MemoryManager memoryManager,
    +			IOManager ioManager,
    +			NetworkEnvironment networkEnvironment,
    +			int numberOfSlots,
     			RpcService rpcService,
    -			HighAvailabilityServices haServices,
    -			ResourceID resourceID) {
    +			HighAvailabilityServices haServices) {
     
     		super(rpcService);
     
    -		this.haServices = checkNotNull(haServices);
    +		this.taskManagerConfig = checkNotNull(taskManagerConfig);
     		this.resourceID = checkNotNull(resourceID);
    +		this.connectionInfo = checkNotNull(connectionInfo);
    +		this.memoryManager = checkNotNull(memoryManager);
    +		this.ioManager = checkNotNull(ioManager);
    +		this.networkEnvironment = checkNotNull(networkEnvironment);
    +		this.numberOfSlots = checkNotNull(numberOfSlots);
    +		this.haServices = checkNotNull(haServices);
    +	}
    +
    +	/**
    +	 * Starts and runs the TaskManager.
    +	 * <p/>
    +	 * This method first tries to select the network interface to use for the TaskManager
    +	 * communication. The network interface is used both for the actor communication
    +	 * (coordination) as well as for the data exchange between task managers. Unless
    +	 * the hostname/interface is explicitly configured in the configuration, this
    +	 * method will try out various interfaces and methods to connect to the JobManager
    +	 * and select the one where the connection attempt is successful.
    +	 * <p/>
    +	 * After selecting the network interface, this method brings up an actor system
    +	 * for the TaskManager and its actors, starts the TaskManager's services
    +	 * (library cache, shuffle network stack, ...), and starts the TaskManager itself.
    +	 *
    +	 * @param configuration    The configuration for the TaskManager.
    +	 * @param taskManagerClass The actor class to instantiate.
    +	 *                         Allows to use TaskManager subclasses for example for YARN.
    +	 */
    +	public static void selectNetworkInterfaceAndRunTaskManager(
    +		Configuration configuration,
    +		ResourceID resourceID,
    +		Class<? extends TaskManager> taskManagerClass) throws Exception {
    +
    +		Tuple2<String, Integer> tuple2 = selectNetworkInterfaceAndPort(configuration);
    +
    +		runTaskManager(tuple2._1(), resourceID, tuple2._2(), configuration, taskManagerClass);
    +	}
    +
    +	private static Tuple2<String, Integer> selectNetworkInterfaceAndPort(Configuration configuration)
    +		throws Exception {
    +		String taskManagerHostname = configuration.getString(ConfigConstants.TASK_MANAGER_HOSTNAME_KEY, null);
    +		if (taskManagerHostname != null) {
    +			LOG.info("Using configured hostname/address for TaskManager: " + taskManagerHostname);
    +		} else {
    +			LeaderRetrievalService leaderRetrievalService = LeaderRetrievalUtils.createLeaderRetrievalService(configuration);
    +			FiniteDuration lookupTimeout = AkkaUtils.getLookupTimeout(configuration);
    +
    +			InetAddress taskManagerAddress = LeaderRetrievalUtils.findConnectingAddress(leaderRetrievalService, lookupTimeout);
    +			taskManagerHostname = taskManagerAddress.getHostName();
    +			LOG.info("TaskManager will use hostname/address '{}' ({}) for communication.",
    +				taskManagerHostname, taskManagerAddress.getHostAddress());
    +		}
    +
    +		// if no task manager port has been configured, use 0 (system will pick any free port)
    +		int actorSystemPort = configuration.getInteger(ConfigConstants.TASK_MANAGER_IPC_PORT_KEY, 0);
    +		if (actorSystemPort < 0 || actorSystemPort > 65535) {
    +			throw new IllegalConfigurationException("Invalid value for '" +
    +				ConfigConstants.TASK_MANAGER_IPC_PORT_KEY +
    +				"' (port for the TaskManager actor system) : " + actorSystemPort +
    +				" - Leave config parameter empty or use 0 to let the system choose a port automatically.");
    +		}
    +
    +		return new Tuple2<>(taskManagerHostname, actorSystemPort);
    +	}
    +
    +	/**
    +	 * Starts and runs the TaskManager. Brings up an actor system for the TaskManager and its
    +	 * actors, starts the TaskManager's services (library cache, shuffle network stack, ...),
    +	 * and starts the TaskManager itself.
    +	 * <p/>
    +	 * This method will also spawn a process reaper for the TaskManager (kill the process if
    +	 * the actor fails) and optionally start the JVM memory logging thread.
    +	 *
    +	 * @param taskManagerHostname The hostname/address of the interface where the actor system
    +	 *                            will communicate.
    +	 * @param resourceID          The id of the resource which the task manager will run on.
    +	 * @param actorSystemPort     The port at which the actor system will communicate.
    +	 * @param configuration       The configuration for the TaskManager.
    +	 * @param taskManagerClass    The actor class to instantiate. Allows the use of TaskManager
    +	 *                            subclasses for example for YARN.
    +	 */
    +	private static void runTaskManager(
    +		String taskManagerHostname,
    +		ResourceID resourceID,
    +		int actorSystemPort,
    +		Configuration configuration,
    +		Class<? extends TaskManager> taskManagerClass) throws Exception {
    +
    +		LOG.info("Starting TaskManager");
    +
    +		// Bring up the TaskManager actor system first, bind it to the given address.
    +
    +		LOG.info("Starting TaskManager actor system at " +
    +			NetUtils.hostAndPortToUrlString(taskManagerHostname, actorSystemPort));
    +
    +		ActorSystem taskManagerSystem;
    +		try {
    +			Tuple2<String, Object> address = new Tuple2<String, Object>(taskManagerHostname, actorSystemPort);
    +			Config akkaConfig = AkkaUtils.getAkkaConfig(configuration, new Some<>(address));
    +			LOG.debug("Using akka configuration\n " + akkaConfig);
    +			taskManagerSystem = AkkaUtils.createActorSystem(akkaConfig);
    +		} catch (Throwable t) {
    +			if (t instanceof org.jboss.netty.channel.ChannelException) {
    +				Throwable cause = t.getCause();
    +				if (cause != null && t.getCause() instanceof java.net.BindException) {
    +					String address = NetUtils.hostAndPortToUrlString(taskManagerHostname, actorSystemPort);
    +					throw new IOException("Unable to bind TaskManager actor system to address " +
    +						address + " - " + cause.getMessage(), t);
    +				}
    +			}
    +			throw new Exception("Could not create TaskManager actor system", t);
    +		}
    +
    +		// start all the TaskManager services (network stack,  library cache, ...)
    +		// and the TaskManager actor
    +		try {
    +			LOG.info("Starting TaskManager actor");
    +			ActorRef taskManagerActor = startTaskManagerComponentsAndActor(
    +				configuration,
    +				resourceID,
    +				taskManagerSystem,
    +				taskManagerHostname,
    +				TASK_MANAGER_NAME,
    +				null,
    +				false,
    +				taskManagerClass);
    +
    +			// start a process reaper that watches the JobManager. If the TaskManager actor dies,
    +			// the process reaper will kill the JVM process (to ensure easy failure detection)
    +			LOG.debug("Starting TaskManager process reaper");
    +
    +			taskManagerSystem.actorOf(
    +				Props.create(ProcessReaper.class, taskManagerActor, LOG, RUNTIME_FAILURE_RETURN_CODE),
    +				"TaskManager_Process_Reaper");
    +
    +			// if desired, start the logging daemon that periodically logs the memory usage information
    +			if (LOG.isInfoEnabled() && configuration.getBoolean(
    +				ConfigConstants.TASK_MANAGER_DEBUG_MEMORY_USAGE_START_LOG_THREAD,
    +				ConfigConstants.DEFAULT_TASK_MANAGER_DEBUG_MEMORY_USAGE_START_LOG_THREAD)) {
    +				LOG.info("Starting periodic memory usage logger");
    +
    +				long interval = configuration.getLong(
    +					ConfigConstants.TASK_MANAGER_DEBUG_MEMORY_USAGE_LOG_INTERVAL_MS,
    +					ConfigConstants.DEFAULT_TASK_MANAGER_DEBUG_MEMORY_USAGE_LOG_INTERVAL_MS);
    +
    +				MemoryLogger logger = new MemoryLogger(LOG, interval, taskManagerSystem);
    +				logger.start();
    +			}
    +
    +			// block until everything is done
    +			taskManagerSystem.awaitTermination();
    +		} catch (Throwable t) {
    +			LOG.error("Error while starting up taskManager", t);
    +			try {
    +				taskManagerSystem.shutdown();
    +			} catch (Throwable tt) {
    +				LOG.warn("Could not cleanly shut down actor system", tt);
    +			}
    +			throw t;
    +		}
    +	}
    +
    +	// --------------------------------------------------------------------------
    +	//  Starting and running the TaskManager
    +	// --------------------------------------------------------------------------
    +
    +	/**
    +	 * @param configuration                 The configuration for the TaskManager.
    +	 * @param resourceID                    The id of the resource which the task manager will run on.
    +	 * @param actorSystem                  The actor system that should run the TaskManager actor.
    +	 * @param taskManagerHostname       The hostname/address that describes the TaskManager's data location.
    +	 * @param taskManagerActorName      Optionally the name of the TaskManager actor. If none is given,
    +	 *                                      the actor will use a random name.
    +	 * @param leaderRetrievalService        Optionally, a leader retrieval service can be provided. If none is given,
    +	 *                                      then a LeaderRetrievalService is constructed from the configuration.
    +	 * @param localTaskManagerCommunication If true, the TaskManager will not initiate the TCP network stack.
    +	 * @param taskManagerClass      The class of the TaskManager actor. May be used to give
    +	 *                                      subclasses that understand additional actor messages.
    +	 * @return An ActorRef to the TaskManager actor.
    +	 * @throws org.apache.flink.configuration.IllegalConfigurationException     Thrown, if the given config contains illegal values.
    +	 * @throws java.io.IOException      Thrown, if any of the I/O components (such as buffer pools,
    +	 *                                       I/O manager, ...) cannot be properly started.
    +	 * @throws java.lang.Exception      Thrown is some other error occurs while parsing the configuration
    +	 *                                      or starting the TaskManager components.
    +	 */
    +	public static ActorRef startTaskManagerComponentsAndActor(
    +		Configuration configuration,
    +		ResourceID resourceID,
    +		ActorSystem actorSystem,
    +		String taskManagerHostname,
    +		String taskManagerActorName,
    +		LeaderRetrievalService leaderRetrievalService,
    +		boolean localTaskManagerCommunication,
    +		Class<? extends TaskManager> taskManagerClass) throws Exception {
    +
    +		Tuple4<TaskManagerConfiguration, NetworkEnvironmentConfiguration, InstanceConnectionInfo, MemoryType> tuple4
    +			= parseTaskManagerConfiguration(configuration, taskManagerHostname, localTaskManagerCommunication);
    +
    +		TaskManagerConfiguration taskManagerConfig = tuple4._1();
    +		NetworkEnvironmentConfiguration netConfig = tuple4._2();
    +		InstanceConnectionInfo connectionInfo = tuple4._3();
    +		MemoryType memType = tuple4._4();
    +
    +		// pre-start checks
    +		checkTempDirs(taskManagerConfig.tmpDirPaths());
    +
    +		ExecutionContext executionContext = ExecutionContexts$.MODULE$.fromExecutor(new ForkJoinPool());
    +
    +		// we start the network first, to make sure it can allocate its buffers first
    +		NetworkEnvironment network = new NetworkEnvironment(executionContext, taskManagerConfig.timeout(), netConfig);
    +
    +		// computing the amount of memory to use depends on how much memory is available
    +		// it strictly needs to happen AFTER the network stack has been initialized
    +
    +		// check if a value has been configured
    +		long configuredMemory = configuration.getLong(ConfigConstants.TASK_MANAGER_MEMORY_SIZE_KEY, -1L);
    +		checkConfigParameter(configuredMemory == -1 || configuredMemory > 0, configuredMemory,
    +			ConfigConstants.TASK_MANAGER_MEMORY_SIZE_KEY,
    +			"MemoryManager needs at least one MB of memory. " +
    +				"If you leave this config parameter empty, the system automatically " +
    +				"pick a fraction of the available memory.");
    +
    +		long memorySize;
    --- End diff --
    
    Would make that one `final`.


---
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.
---

[GitHub] flink pull request #2400: [FLINK-4363] Implement TaskManager basic startup o...

Posted by mxm <gi...@git.apache.org>.
Github user mxm commented on a diff in the pull request:

    https://github.com/apache/flink/pull/2400#discussion_r75648406
  
    --- Diff: flink-runtime/src/main/java/org/apache/flink/runtime/rpc/taskexecutor/TaskExecutor.java ---
    @@ -35,27 +79,634 @@
      */
     public class TaskExecutor extends RpcEndpoint<TaskExecutorGateway> {
     
    -	/** The unique resource ID of this TaskExecutor */
    +	private static final Logger LOG = LoggerFactory.getLogger(TaskExecutor.class);
    +
    +	/** Return code for critical errors during the runtime */
    +	private static final int RUNTIME_FAILURE_RETURN_CODE = 2;
    +
    +	/** The name of the TaskManager actor */
    +	private static final String TASK_MANAGER_NAME = "taskmanager";
    +
    +	/** The unique resource ID of this TaskManager */
     	private final ResourceID resourceID;
     
     	/** The access to the leader election and metadata storage services */
     	private final HighAvailabilityServices haServices;
     
    -	// --------- resource manager --------
    +	/** The task manager configuration */
    +	private final TaskManagerConfiguration taskManagerConfig;
    +
    +	/** The connection information of the task manager */
    +	private final InstanceConnectionInfo connectionInfo;
    +
    +	/** The I/O manager component in the task manager */
    +	private final IOManager ioManager;
    +
    +	/** The memory manager component in the task manager */
    +	private final MemoryManager memoryManager;
    +
    +	/** The network component in the task manager */
    +	private final NetworkEnvironment networkEnvironment;
    +
    +	/** The number of slots in the task manager, should be 1 for YARN */
    +	private final int numberOfSlots;
     
    +	// --------- resource manager --------
     	private TaskExecutorToResourceManagerConnection resourceManagerConnection;
     
     	// ------------------------------------------------------------------------
     
     	public TaskExecutor(
    +			TaskManagerConfiguration taskManagerConfig,
    +			ResourceID resourceID,
    +			InstanceConnectionInfo connectionInfo,
    +			MemoryManager memoryManager,
    +			IOManager ioManager,
    +			NetworkEnvironment networkEnvironment,
    +			int numberOfSlots,
     			RpcService rpcService,
    -			HighAvailabilityServices haServices,
    -			ResourceID resourceID) {
    +			HighAvailabilityServices haServices) {
     
     		super(rpcService);
     
    -		this.haServices = checkNotNull(haServices);
    +		this.taskManagerConfig = checkNotNull(taskManagerConfig);
     		this.resourceID = checkNotNull(resourceID);
    +		this.connectionInfo = checkNotNull(connectionInfo);
    +		this.memoryManager = checkNotNull(memoryManager);
    +		this.ioManager = checkNotNull(ioManager);
    +		this.networkEnvironment = checkNotNull(networkEnvironment);
    +		this.numberOfSlots = checkNotNull(numberOfSlots);
    +		this.haServices = checkNotNull(haServices);
    +	}
    +
    +	/**
    +	 * Starts and runs the TaskManager.
    +	 * <p/>
    +	 * This method first tries to select the network interface to use for the TaskManager
    +	 * communication. The network interface is used both for the actor communication
    +	 * (coordination) as well as for the data exchange between task managers. Unless
    +	 * the hostname/interface is explicitly configured in the configuration, this
    +	 * method will try out various interfaces and methods to connect to the JobManager
    +	 * and select the one where the connection attempt is successful.
    +	 * <p/>
    +	 * After selecting the network interface, this method brings up an actor system
    +	 * for the TaskManager and its actors, starts the TaskManager's services
    +	 * (library cache, shuffle network stack, ...), and starts the TaskManager itself.
    +	 *
    +	 * @param configuration    The configuration for the TaskManager.
    +	 * @param taskManagerClass The actor class to instantiate.
    +	 *                         Allows to use TaskManager subclasses for example for YARN.
    +	 */
    +	public static void selectNetworkInterfaceAndRunTaskManager(
    +		Configuration configuration,
    +		ResourceID resourceID,
    +		Class<? extends TaskManager> taskManagerClass) throws Exception {
    +
    +		Tuple2<String, Integer> tuple2 = selectNetworkInterfaceAndPort(configuration);
    +
    +		runTaskManager(tuple2._1(), resourceID, tuple2._2(), configuration, taskManagerClass);
    +	}
    +
    +	private static Tuple2<String, Integer> selectNetworkInterfaceAndPort(Configuration configuration)
    +		throws Exception {
    +		String taskManagerHostname = configuration.getString(ConfigConstants.TASK_MANAGER_HOSTNAME_KEY, null);
    +		if (taskManagerHostname != null) {
    +			LOG.info("Using configured hostname/address for TaskManager: " + taskManagerHostname);
    +		} else {
    +			LeaderRetrievalService leaderRetrievalService = LeaderRetrievalUtils.createLeaderRetrievalService(configuration);
    +			FiniteDuration lookupTimeout = AkkaUtils.getLookupTimeout(configuration);
    +
    +			InetAddress taskManagerAddress = LeaderRetrievalUtils.findConnectingAddress(leaderRetrievalService, lookupTimeout);
    +			taskManagerHostname = taskManagerAddress.getHostName();
    +			LOG.info("TaskManager will use hostname/address '{}' ({}) for communication.",
    +				taskManagerHostname, taskManagerAddress.getHostAddress());
    +		}
    +
    +		// if no task manager port has been configured, use 0 (system will pick any free port)
    +		int actorSystemPort = configuration.getInteger(ConfigConstants.TASK_MANAGER_IPC_PORT_KEY, 0);
    +		if (actorSystemPort < 0 || actorSystemPort > 65535) {
    +			throw new IllegalConfigurationException("Invalid value for '" +
    +				ConfigConstants.TASK_MANAGER_IPC_PORT_KEY +
    +				"' (port for the TaskManager actor system) : " + actorSystemPort +
    +				" - Leave config parameter empty or use 0 to let the system choose a port automatically.");
    +		}
    +
    +		return new Tuple2<>(taskManagerHostname, actorSystemPort);
    +	}
    +
    +	/**
    +	 * Starts and runs the TaskManager. Brings up an actor system for the TaskManager and its
    +	 * actors, starts the TaskManager's services (library cache, shuffle network stack, ...),
    +	 * and starts the TaskManager itself.
    +	 * <p/>
    +	 * This method will also spawn a process reaper for the TaskManager (kill the process if
    +	 * the actor fails) and optionally start the JVM memory logging thread.
    +	 *
    +	 * @param taskManagerHostname The hostname/address of the interface where the actor system
    +	 *                            will communicate.
    +	 * @param resourceID          The id of the resource which the task manager will run on.
    +	 * @param actorSystemPort     The port at which the actor system will communicate.
    +	 * @param configuration       The configuration for the TaskManager.
    +	 * @param taskManagerClass    The actor class to instantiate. Allows the use of TaskManager
    +	 *                            subclasses for example for YARN.
    +	 */
    +	private static void runTaskManager(
    +		String taskManagerHostname,
    +		ResourceID resourceID,
    +		int actorSystemPort,
    +		Configuration configuration,
    +		Class<? extends TaskManager> taskManagerClass) throws Exception {
    +
    +		LOG.info("Starting TaskManager");
    +
    +		// Bring up the TaskManager actor system first, bind it to the given address.
    +
    +		LOG.info("Starting TaskManager actor system at " +
    +			NetUtils.hostAndPortToUrlString(taskManagerHostname, actorSystemPort));
    +
    +		ActorSystem taskManagerSystem;
    +		try {
    +			Tuple2<String, Object> address = new Tuple2<String, Object>(taskManagerHostname, actorSystemPort);
    +			Config akkaConfig = AkkaUtils.getAkkaConfig(configuration, new Some<>(address));
    +			LOG.debug("Using akka configuration\n " + akkaConfig);
    +			taskManagerSystem = AkkaUtils.createActorSystem(akkaConfig);
    +		} catch (Throwable t) {
    +			if (t instanceof org.jboss.netty.channel.ChannelException) {
    +				Throwable cause = t.getCause();
    +				if (cause != null && t.getCause() instanceof java.net.BindException) {
    +					String address = NetUtils.hostAndPortToUrlString(taskManagerHostname, actorSystemPort);
    +					throw new IOException("Unable to bind TaskManager actor system to address " +
    +						address + " - " + cause.getMessage(), t);
    +				}
    +			}
    +			throw new Exception("Could not create TaskManager actor system", t);
    +		}
    +
    +		// start all the TaskManager services (network stack,  library cache, ...)
    +		// and the TaskManager actor
    +		try {
    +			LOG.info("Starting TaskManager actor");
    +			ActorRef taskManagerActor = startTaskManagerComponentsAndActor(
    +				configuration,
    +				resourceID,
    +				taskManagerSystem,
    +				taskManagerHostname,
    +				TASK_MANAGER_NAME,
    +				null,
    +				false,
    +				taskManagerClass);
    +
    +			// start a process reaper that watches the JobManager. If the TaskManager actor dies,
    +			// the process reaper will kill the JVM process (to ensure easy failure detection)
    +			LOG.debug("Starting TaskManager process reaper");
    +
    +			taskManagerSystem.actorOf(
    +				Props.create(ProcessReaper.class, taskManagerActor, LOG, RUNTIME_FAILURE_RETURN_CODE),
    +				"TaskManager_Process_Reaper");
    +
    +			// if desired, start the logging daemon that periodically logs the memory usage information
    +			if (LOG.isInfoEnabled() && configuration.getBoolean(
    +				ConfigConstants.TASK_MANAGER_DEBUG_MEMORY_USAGE_START_LOG_THREAD,
    +				ConfigConstants.DEFAULT_TASK_MANAGER_DEBUG_MEMORY_USAGE_START_LOG_THREAD)) {
    +				LOG.info("Starting periodic memory usage logger");
    +
    +				long interval = configuration.getLong(
    +					ConfigConstants.TASK_MANAGER_DEBUG_MEMORY_USAGE_LOG_INTERVAL_MS,
    +					ConfigConstants.DEFAULT_TASK_MANAGER_DEBUG_MEMORY_USAGE_LOG_INTERVAL_MS);
    +
    +				MemoryLogger logger = new MemoryLogger(LOG, interval, taskManagerSystem);
    +				logger.start();
    +			}
    +
    +			// block until everything is done
    +			taskManagerSystem.awaitTermination();
    +		} catch (Throwable t) {
    +			LOG.error("Error while starting up taskManager", t);
    +			try {
    +				taskManagerSystem.shutdown();
    +			} catch (Throwable tt) {
    +				LOG.warn("Could not cleanly shut down actor system", tt);
    +			}
    +			throw t;
    +		}
    +	}
    +
    +	// --------------------------------------------------------------------------
    +	//  Starting and running the TaskManager
    +	// --------------------------------------------------------------------------
    +
    +	/**
    +	 * @param configuration                 The configuration for the TaskManager.
    +	 * @param resourceID                    The id of the resource which the task manager will run on.
    +	 * @param actorSystem                  The actor system that should run the TaskManager actor.
    +	 * @param taskManagerHostname       The hostname/address that describes the TaskManager's data location.
    +	 * @param taskManagerActorName      Optionally the name of the TaskManager actor. If none is given,
    +	 *                                      the actor will use a random name.
    +	 * @param leaderRetrievalService        Optionally, a leader retrieval service can be provided. If none is given,
    +	 *                                      then a LeaderRetrievalService is constructed from the configuration.
    +	 * @param localTaskManagerCommunication If true, the TaskManager will not initiate the TCP network stack.
    +	 * @param taskManagerClass      The class of the TaskManager actor. May be used to give
    +	 *                                      subclasses that understand additional actor messages.
    +	 * @return An ActorRef to the TaskManager actor.
    +	 * @throws org.apache.flink.configuration.IllegalConfigurationException     Thrown, if the given config contains illegal values.
    +	 * @throws java.io.IOException      Thrown, if any of the I/O components (such as buffer pools,
    +	 *                                       I/O manager, ...) cannot be properly started.
    +	 * @throws java.lang.Exception      Thrown is some other error occurs while parsing the configuration
    +	 *                                      or starting the TaskManager components.
    +	 */
    +	public static ActorRef startTaskManagerComponentsAndActor(
    +		Configuration configuration,
    +		ResourceID resourceID,
    +		ActorSystem actorSystem,
    +		String taskManagerHostname,
    +		String taskManagerActorName,
    +		LeaderRetrievalService leaderRetrievalService,
    +		boolean localTaskManagerCommunication,
    +		Class<? extends TaskManager> taskManagerClass) throws Exception {
    +
    +		Tuple4<TaskManagerConfiguration, NetworkEnvironmentConfiguration, InstanceConnectionInfo, MemoryType> tuple4
    +			= parseTaskManagerConfiguration(configuration, taskManagerHostname, localTaskManagerCommunication);
    +
    +		TaskManagerConfiguration taskManagerConfig = tuple4._1();
    +		NetworkEnvironmentConfiguration netConfig = tuple4._2();
    +		InstanceConnectionInfo connectionInfo = tuple4._3();
    +		MemoryType memType = tuple4._4();
    +
    +		// pre-start checks
    +		checkTempDirs(taskManagerConfig.tmpDirPaths());
    +
    +		ExecutionContext executionContext = ExecutionContexts$.MODULE$.fromExecutor(new ForkJoinPool());
    +
    +		// we start the network first, to make sure it can allocate its buffers first
    +		NetworkEnvironment network = new NetworkEnvironment(executionContext, taskManagerConfig.timeout(), netConfig);
    +
    +		// computing the amount of memory to use depends on how much memory is available
    +		// it strictly needs to happen AFTER the network stack has been initialized
    +
    +		// check if a value has been configured
    +		long configuredMemory = configuration.getLong(ConfigConstants.TASK_MANAGER_MEMORY_SIZE_KEY, -1L);
    +		checkConfigParameter(configuredMemory == -1 || configuredMemory > 0, configuredMemory,
    +			ConfigConstants.TASK_MANAGER_MEMORY_SIZE_KEY,
    +			"MemoryManager needs at least one MB of memory. " +
    +				"If you leave this config parameter empty, the system automatically " +
    +				"pick a fraction of the available memory.");
    +
    +		long memorySize;
    +		boolean preAllocateMemory = configuration.getBoolean(
    +			ConfigConstants.TASK_MANAGER_MEMORY_PRE_ALLOCATE_KEY,
    +			ConfigConstants.DEFAULT_TASK_MANAGER_MEMORY_PRE_ALLOCATE);
    +		if (configuredMemory > 0) {
    +			if (preAllocateMemory) {
    +				LOG.info("Using {} MB for managed memory." , configuredMemory);
    +			} else {
    +				LOG.info("Limiting managed memory to {} MB, memory will be allocated lazily." , configuredMemory);
    +			}
    +			memorySize = configuredMemory << 20; // megabytes to bytes
    +		} else {
    +			float fraction = configuration.getFloat(
    +				ConfigConstants.TASK_MANAGER_MEMORY_FRACTION_KEY,
    +				ConfigConstants.DEFAULT_MEMORY_MANAGER_MEMORY_FRACTION);
    +			checkConfigParameter(fraction > 0.0f && fraction < 1.0f, fraction,
    +				ConfigConstants.TASK_MANAGER_MEMORY_FRACTION_KEY,
    +				"MemoryManager fraction of the free memory must be between 0.0 and 1.0");
    +
    +			if (memType == MemoryType.HEAP) {
    +				long relativeMemSize = (long) (EnvironmentInformation.getSizeOfFreeHeapMemoryWithDefrag() * fraction);
    +				if (preAllocateMemory) {
    +					LOG.info("Using {} of the currently free heap space for managed heap memory ({} MB)." ,
    +						fraction , relativeMemSize >> 20);
    +				} else {
    +					LOG.info("Limiting managed memory to {} of the currently free heap space ({} MB), " +
    +						"memory will be allocated lazily." , fraction , relativeMemSize >> 20);
    +				}
    +				memorySize = relativeMemSize;
    +			} else if (memType == MemoryType.OFF_HEAP) {
    +				// The maximum heap memory has been adjusted according to the fraction
    +				long maxMemory = EnvironmentInformation.getMaxJvmHeapMemory();
    +				long directMemorySize = (long) (maxMemory / (1.0 - fraction) * fraction);
    +				if (preAllocateMemory) {
    +					LOG.info("Using {} of the maximum memory size for managed off-heap memory ({} MB)." ,
    +						fraction, directMemorySize >> 20);
    +				} else {
    +					LOG.info("Limiting managed memory to {} of the maximum memory size ({} MB)," +
    +						" memory will be allocated lazily.", fraction, directMemorySize >> 20);
    +				}
    +				memorySize = directMemorySize;
    +			} else {
    +				throw new RuntimeException("No supported memory type detected.");
    +			}
    +		}
    +
    +		// now start the memory manager
    +		MemoryManager memoryManager;
    +		try {
    +			memoryManager = new MemoryManager(
    +				memorySize,
    +				taskManagerConfig.numberOfSlots(),
    +				netConfig.networkBufferSize(),
    +				memType,
    +				preAllocateMemory);
    +		} catch (OutOfMemoryError e) {
    +			if (memType == MemoryType.HEAP) {
    +				throw new Exception("OutOfMemory error (" + e.getMessage() +
    +					") while allocating the TaskManager heap memory (" + memorySize + " bytes).", e);
    +			} else if (memType == MemoryType.OFF_HEAP) {
    +				throw new Exception("OutOfMemory error (" + e.getMessage() +
    +					") while allocating the TaskManager off-heap memory (" + memorySize +
    +					" bytes).Try increasing the maximum direct memory (-XX:MaxDirectMemorySize)", e);
    +			} else {
    +				throw e;
    +			}
    +		}
    +
    +		// start the I/O manager, it will create some temp directories.
    +		IOManager ioManager = new IOManagerAsync(taskManagerConfig.tmpDirPaths());
    +
    +		if (leaderRetrievalService == null){
    +			leaderRetrievalService = LeaderRetrievalUtils.createLeaderRetrievalService(configuration);
    +		}
    +
    +		// create the actor properties (which define the actor constructor parameters)
    +		Props tmProps = Props.create(
    +			taskManagerClass,
    +			taskManagerConfig,
    +			resourceID,
    +			connectionInfo,
    +			memoryManager,
    +			ioManager,
    +			network,
    +			taskManagerConfig.numberOfSlots(),
    +			leaderRetrievalService);
    +
    +		ActorRef taskManagerActorRef;
    +		if (taskManagerActorName != null && !taskManagerActorName.equals("")) {
    +			taskManagerActorRef = actorSystem.actorOf(tmProps, taskManagerActorName);
    +		} else {
    +			taskManagerActorRef = actorSystem.actorOf(tmProps);
    +		}
    +
    +		return taskManagerActorRef;
    +	}
    +
    +	// --------------------------------------------------------------------------
    +	//  Parsing and checking the TaskManager Configuration
    +	// --------------------------------------------------------------------------
    +
    +	/**
    +	 * Utility method to extract TaskManager config parameters from the configuration and to
    +	 * sanity check them.
    +	 *
    +	 * @param configuration                 The configuration.
    +	 * @param taskManagerHostname           The host name under which the TaskManager communicates.
    +	 * @param localTaskManagerCommunication True, to skip initializing the network stack.
    +	 *                                      Use only in cases where only one task manager runs.
    +	 * @return A tuple (TaskManagerConfiguration, network configuration,
    +	 * InstanceConnectionInfo, JobManager actor Akka URL).
    +	 */
    +	private static Tuple4<TaskManagerConfiguration, NetworkEnvironmentConfiguration, InstanceConnectionInfo, MemoryType>
    +		parseTaskManagerConfiguration(Configuration configuration, String taskManagerHostname, boolean localTaskManagerCommunication)
    +		throws Exception {
    +
    +		// ------- read values from the config and check them ---------
    +		//                      (a lot of them)
    +
    +		// ----> hosts / ports for communication and data exchange
    +
    +		int dataport = configuration.getInteger(ConfigConstants.TASK_MANAGER_DATA_PORT_KEY,
    +			ConfigConstants.DEFAULT_TASK_MANAGER_DATA_PORT);
    +		if (dataport == 0) {
    +			dataport = NetUtils.getAvailablePort();
    +		}
    +		checkConfigParameter(dataport > 0, dataport, ConfigConstants.TASK_MANAGER_DATA_PORT_KEY,
    +			"Leave config parameter empty or use 0 to let the system choose a port automatically.");
    +
    +		InetAddress taskManagerAddress = InetAddress.getByName(taskManagerHostname);
    +		InstanceConnectionInfo connectionInfo = new InstanceConnectionInfo(taskManagerAddress, dataport);
    +
    +		// ----> memory / network stack (shuffles/broadcasts), task slots, temp directories
    +
    +		// we need this because many configs have been written with a "-1" entry
    +		int slots = configuration.getInteger(ConfigConstants.TASK_MANAGER_NUM_TASK_SLOTS, 1);
    +		if (slots == -1) {
    +			slots = 1;
    +		}
    +		checkConfigParameter(slots >= 1, slots, ConfigConstants.TASK_MANAGER_NUM_TASK_SLOTS,
    +			"Number of task slots must be at least one.");
    +
    +		int numNetworkBuffers = configuration.getInteger(
    +			ConfigConstants.TASK_MANAGER_NETWORK_NUM_BUFFERS_KEY,
    +			ConfigConstants.DEFAULT_TASK_MANAGER_NETWORK_NUM_BUFFERS);
    +		checkConfigParameter(numNetworkBuffers > 0, numNetworkBuffers,
    +			ConfigConstants.TASK_MANAGER_NETWORK_NUM_BUFFERS_KEY, "");
    +
    +		int pageSize = configuration.getInteger(
    +			ConfigConstants.TASK_MANAGER_MEMORY_SEGMENT_SIZE_KEY,
    +			ConfigConstants.DEFAULT_TASK_MANAGER_MEMORY_SEGMENT_SIZE);
    +		// check page size of for minimum size
    +		checkConfigParameter(pageSize >= MemoryManager.MIN_PAGE_SIZE, pageSize,
    +			ConfigConstants.TASK_MANAGER_MEMORY_SEGMENT_SIZE_KEY,
    +			"Minimum memory segment size is " + MemoryManager.MIN_PAGE_SIZE);
    +		// check page size for power of two
    +		checkConfigParameter(MathUtils.isPowerOf2(pageSize), pageSize,
    +			ConfigConstants.TASK_MANAGER_MEMORY_SEGMENT_SIZE_KEY,
    +			"Memory segment size must be a power of 2.");
    +
    +		// check whether we use heap or off-heap memory
    +		MemoryType memType;
    +		if (configuration.getBoolean(ConfigConstants.TASK_MANAGER_MEMORY_OFF_HEAP_KEY, false)) {
    +			memType = MemoryType.OFF_HEAP;
    +		} else {
    +			memType = MemoryType.HEAP;
    +		}
    +
    +		// initialize the memory segment factory accordingly
    +		if (memType == MemoryType.HEAP) {
    +			if (!MemorySegmentFactory.isInitialized()) {
    +				MemorySegmentFactory.initializeFactory(HeapMemorySegment.FACTORY);
    +			} else if (MemorySegmentFactory.getFactory() != HeapMemorySegment.FACTORY) {
    +				throw new Exception("Memory type is set to heap memory, but memory segment " +
    +					"factory has been initialized for off-heap memory segments");
    +			}
    +		} else {
    +			if (!MemorySegmentFactory.isInitialized()) {
    +				MemorySegmentFactory.initializeFactory(HybridMemorySegment.FACTORY);
    +			} else if (MemorySegmentFactory.getFactory() != HybridMemorySegment.FACTORY) {
    +				throw new Exception("Memory type is set to off-heap memory, but memory segment " +
    +					"factory has been initialized for heap memory segments");
    +			}
    +		}
    +
    +		String[] tmpDirs = configuration.getString(
    +			ConfigConstants.TASK_MANAGER_TMP_DIR_KEY,
    +			ConfigConstants.DEFAULT_TASK_MANAGER_TMP_PATH).split(",|" + File.pathSeparator);
    +
    +		NettyConfig nettyConfig = null;
    --- End diff --
    
    `final` and no need to assign `null`.


---
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.
---

[GitHub] flink pull request #2400: [FLINK-4363] Implement TaskManager basic startup o...

Posted by mxm <gi...@git.apache.org>.
Github user mxm commented on a diff in the pull request:

    https://github.com/apache/flink/pull/2400#discussion_r75680917
  
    --- Diff: flink-runtime/src/main/java/org/apache/flink/runtime/rpc/taskexecutor/TaskExecutor.java ---
    @@ -35,27 +79,634 @@
      */
     public class TaskExecutor extends RpcEndpoint<TaskExecutorGateway> {
     
    -	/** The unique resource ID of this TaskExecutor */
    +	private static final Logger LOG = LoggerFactory.getLogger(TaskExecutor.class);
    +
    +	/** Return code for critical errors during the runtime */
    +	private static final int RUNTIME_FAILURE_RETURN_CODE = 2;
    +
    +	/** The name of the TaskManager actor */
    +	private static final String TASK_MANAGER_NAME = "taskmanager";
    +
    +	/** The unique resource ID of this TaskManager */
     	private final ResourceID resourceID;
     
     	/** The access to the leader election and metadata storage services */
     	private final HighAvailabilityServices haServices;
     
    -	// --------- resource manager --------
    +	/** The task manager configuration */
    +	private final TaskManagerConfiguration taskManagerConfig;
    +
    +	/** The connection information of the task manager */
    +	private final InstanceConnectionInfo connectionInfo;
    +
    +	/** The I/O manager component in the task manager */
    +	private final IOManager ioManager;
    +
    +	/** The memory manager component in the task manager */
    +	private final MemoryManager memoryManager;
    +
    +	/** The network component in the task manager */
    +	private final NetworkEnvironment networkEnvironment;
    +
    +	/** The number of slots in the task manager, should be 1 for YARN */
    +	private final int numberOfSlots;
     
    +	// --------- resource manager --------
     	private TaskExecutorToResourceManagerConnection resourceManagerConnection;
     
     	// ------------------------------------------------------------------------
     
     	public TaskExecutor(
    +			TaskManagerConfiguration taskManagerConfig,
    +			ResourceID resourceID,
    +			InstanceConnectionInfo connectionInfo,
    +			MemoryManager memoryManager,
    +			IOManager ioManager,
    +			NetworkEnvironment networkEnvironment,
    +			int numberOfSlots,
     			RpcService rpcService,
    -			HighAvailabilityServices haServices,
    -			ResourceID resourceID) {
    +			HighAvailabilityServices haServices) {
     
     		super(rpcService);
     
    -		this.haServices = checkNotNull(haServices);
    +		this.taskManagerConfig = checkNotNull(taskManagerConfig);
     		this.resourceID = checkNotNull(resourceID);
    +		this.connectionInfo = checkNotNull(connectionInfo);
    +		this.memoryManager = checkNotNull(memoryManager);
    +		this.ioManager = checkNotNull(ioManager);
    +		this.networkEnvironment = checkNotNull(networkEnvironment);
    +		this.numberOfSlots = checkNotNull(numberOfSlots);
    +		this.haServices = checkNotNull(haServices);
    +	}
    +
    +	/**
    +	 * Starts and runs the TaskManager.
    +	 * <p/>
    +	 * This method first tries to select the network interface to use for the TaskManager
    +	 * communication. The network interface is used both for the actor communication
    +	 * (coordination) as well as for the data exchange between task managers. Unless
    +	 * the hostname/interface is explicitly configured in the configuration, this
    +	 * method will try out various interfaces and methods to connect to the JobManager
    +	 * and select the one where the connection attempt is successful.
    +	 * <p/>
    +	 * After selecting the network interface, this method brings up an actor system
    +	 * for the TaskManager and its actors, starts the TaskManager's services
    +	 * (library cache, shuffle network stack, ...), and starts the TaskManager itself.
    +	 *
    +	 * @param configuration    The configuration for the TaskManager.
    +	 * @param taskManagerClass The actor class to instantiate.
    +	 *                         Allows to use TaskManager subclasses for example for YARN.
    +	 */
    +	public static void selectNetworkInterfaceAndRunTaskManager(
    +		Configuration configuration,
    +		ResourceID resourceID,
    +		Class<? extends TaskManager> taskManagerClass) throws Exception {
    +
    +		Tuple2<String, Integer> tuple2 = selectNetworkInterfaceAndPort(configuration);
    +
    +		runTaskManager(tuple2._1(), resourceID, tuple2._2(), configuration, taskManagerClass);
    +	}
    +
    +	private static Tuple2<String, Integer> selectNetworkInterfaceAndPort(Configuration configuration)
    +		throws Exception {
    +		String taskManagerHostname = configuration.getString(ConfigConstants.TASK_MANAGER_HOSTNAME_KEY, null);
    +		if (taskManagerHostname != null) {
    +			LOG.info("Using configured hostname/address for TaskManager: " + taskManagerHostname);
    +		} else {
    +			LeaderRetrievalService leaderRetrievalService = LeaderRetrievalUtils.createLeaderRetrievalService(configuration);
    +			FiniteDuration lookupTimeout = AkkaUtils.getLookupTimeout(configuration);
    +
    +			InetAddress taskManagerAddress = LeaderRetrievalUtils.findConnectingAddress(leaderRetrievalService, lookupTimeout);
    +			taskManagerHostname = taskManagerAddress.getHostName();
    +			LOG.info("TaskManager will use hostname/address '{}' ({}) for communication.",
    +				taskManagerHostname, taskManagerAddress.getHostAddress());
    +		}
    +
    +		// if no task manager port has been configured, use 0 (system will pick any free port)
    +		int actorSystemPort = configuration.getInteger(ConfigConstants.TASK_MANAGER_IPC_PORT_KEY, 0);
    +		if (actorSystemPort < 0 || actorSystemPort > 65535) {
    +			throw new IllegalConfigurationException("Invalid value for '" +
    +				ConfigConstants.TASK_MANAGER_IPC_PORT_KEY +
    +				"' (port for the TaskManager actor system) : " + actorSystemPort +
    +				" - Leave config parameter empty or use 0 to let the system choose a port automatically.");
    +		}
    +
    +		return new Tuple2<>(taskManagerHostname, actorSystemPort);
    +	}
    +
    +	/**
    +	 * Starts and runs the TaskManager. Brings up an actor system for the TaskManager and its
    +	 * actors, starts the TaskManager's services (library cache, shuffle network stack, ...),
    +	 * and starts the TaskManager itself.
    +	 * <p/>
    +	 * This method will also spawn a process reaper for the TaskManager (kill the process if
    +	 * the actor fails) and optionally start the JVM memory logging thread.
    +	 *
    +	 * @param taskManagerHostname The hostname/address of the interface where the actor system
    +	 *                            will communicate.
    +	 * @param resourceID          The id of the resource which the task manager will run on.
    +	 * @param actorSystemPort     The port at which the actor system will communicate.
    +	 * @param configuration       The configuration for the TaskManager.
    +	 * @param taskManagerClass    The actor class to instantiate. Allows the use of TaskManager
    +	 *                            subclasses for example for YARN.
    +	 */
    +	private static void runTaskManager(
    +		String taskManagerHostname,
    +		ResourceID resourceID,
    +		int actorSystemPort,
    +		Configuration configuration,
    +		Class<? extends TaskManager> taskManagerClass) throws Exception {
    +
    +		LOG.info("Starting TaskManager");
    +
    +		// Bring up the TaskManager actor system first, bind it to the given address.
    +
    +		LOG.info("Starting TaskManager actor system at " +
    +			NetUtils.hostAndPortToUrlString(taskManagerHostname, actorSystemPort));
    +
    +		ActorSystem taskManagerSystem;
    +		try {
    +			Tuple2<String, Object> address = new Tuple2<String, Object>(taskManagerHostname, actorSystemPort);
    +			Config akkaConfig = AkkaUtils.getAkkaConfig(configuration, new Some<>(address));
    +			LOG.debug("Using akka configuration\n " + akkaConfig);
    +			taskManagerSystem = AkkaUtils.createActorSystem(akkaConfig);
    +		} catch (Throwable t) {
    +			if (t instanceof org.jboss.netty.channel.ChannelException) {
    +				Throwable cause = t.getCause();
    +				if (cause != null && t.getCause() instanceof java.net.BindException) {
    +					String address = NetUtils.hostAndPortToUrlString(taskManagerHostname, actorSystemPort);
    +					throw new IOException("Unable to bind TaskManager actor system to address " +
    +						address + " - " + cause.getMessage(), t);
    +				}
    +			}
    +			throw new Exception("Could not create TaskManager actor system", t);
    +		}
    +
    +		// start all the TaskManager services (network stack,  library cache, ...)
    +		// and the TaskManager actor
    +		try {
    +			LOG.info("Starting TaskManager actor");
    +			ActorRef taskManagerActor = startTaskManagerComponentsAndActor(
    +				configuration,
    +				resourceID,
    +				taskManagerSystem,
    +				taskManagerHostname,
    +				TASK_MANAGER_NAME,
    +				null,
    +				false,
    +				taskManagerClass);
    +
    +			// start a process reaper that watches the JobManager. If the TaskManager actor dies,
    +			// the process reaper will kill the JVM process (to ensure easy failure detection)
    +			LOG.debug("Starting TaskManager process reaper");
    +
    +			taskManagerSystem.actorOf(
    +				Props.create(ProcessReaper.class, taskManagerActor, LOG, RUNTIME_FAILURE_RETURN_CODE),
    +				"TaskManager_Process_Reaper");
    +
    +			// if desired, start the logging daemon that periodically logs the memory usage information
    +			if (LOG.isInfoEnabled() && configuration.getBoolean(
    +				ConfigConstants.TASK_MANAGER_DEBUG_MEMORY_USAGE_START_LOG_THREAD,
    +				ConfigConstants.DEFAULT_TASK_MANAGER_DEBUG_MEMORY_USAGE_START_LOG_THREAD)) {
    +				LOG.info("Starting periodic memory usage logger");
    +
    +				long interval = configuration.getLong(
    +					ConfigConstants.TASK_MANAGER_DEBUG_MEMORY_USAGE_LOG_INTERVAL_MS,
    +					ConfigConstants.DEFAULT_TASK_MANAGER_DEBUG_MEMORY_USAGE_LOG_INTERVAL_MS);
    +
    +				MemoryLogger logger = new MemoryLogger(LOG, interval, taskManagerSystem);
    +				logger.start();
    +			}
    +
    +			// block until everything is done
    +			taskManagerSystem.awaitTermination();
    +		} catch (Throwable t) {
    +			LOG.error("Error while starting up taskManager", t);
    +			try {
    +				taskManagerSystem.shutdown();
    +			} catch (Throwable tt) {
    +				LOG.warn("Could not cleanly shut down actor system", tt);
    +			}
    +			throw t;
    +		}
    +	}
    +
    +	// --------------------------------------------------------------------------
    +	//  Starting and running the TaskManager
    +	// --------------------------------------------------------------------------
    +
    +	/**
    +	 * @param configuration                 The configuration for the TaskManager.
    +	 * @param resourceID                    The id of the resource which the task manager will run on.
    +	 * @param actorSystem                  The actor system that should run the TaskManager actor.
    +	 * @param taskManagerHostname       The hostname/address that describes the TaskManager's data location.
    +	 * @param taskManagerActorName      Optionally the name of the TaskManager actor. If none is given,
    +	 *                                      the actor will use a random name.
    +	 * @param leaderRetrievalService        Optionally, a leader retrieval service can be provided. If none is given,
    +	 *                                      then a LeaderRetrievalService is constructed from the configuration.
    +	 * @param localTaskManagerCommunication If true, the TaskManager will not initiate the TCP network stack.
    +	 * @param taskManagerClass      The class of the TaskManager actor. May be used to give
    +	 *                                      subclasses that understand additional actor messages.
    +	 * @return An ActorRef to the TaskManager actor.
    +	 * @throws org.apache.flink.configuration.IllegalConfigurationException     Thrown, if the given config contains illegal values.
    +	 * @throws java.io.IOException      Thrown, if any of the I/O components (such as buffer pools,
    +	 *                                       I/O manager, ...) cannot be properly started.
    +	 * @throws java.lang.Exception      Thrown is some other error occurs while parsing the configuration
    +	 *                                      or starting the TaskManager components.
    +	 */
    +	public static ActorRef startTaskManagerComponentsAndActor(
    +		Configuration configuration,
    +		ResourceID resourceID,
    +		ActorSystem actorSystem,
    +		String taskManagerHostname,
    +		String taskManagerActorName,
    +		LeaderRetrievalService leaderRetrievalService,
    +		boolean localTaskManagerCommunication,
    +		Class<? extends TaskManager> taskManagerClass) throws Exception {
    +
    +		Tuple4<TaskManagerConfiguration, NetworkEnvironmentConfiguration, InstanceConnectionInfo, MemoryType> tuple4
    +			= parseTaskManagerConfiguration(configuration, taskManagerHostname, localTaskManagerCommunication);
    +
    +		TaskManagerConfiguration taskManagerConfig = tuple4._1();
    +		NetworkEnvironmentConfiguration netConfig = tuple4._2();
    +		InstanceConnectionInfo connectionInfo = tuple4._3();
    +		MemoryType memType = tuple4._4();
    +
    +		// pre-start checks
    +		checkTempDirs(taskManagerConfig.tmpDirPaths());
    +
    +		ExecutionContext executionContext = ExecutionContexts$.MODULE$.fromExecutor(new ForkJoinPool());
    +
    +		// we start the network first, to make sure it can allocate its buffers first
    +		NetworkEnvironment network = new NetworkEnvironment(executionContext, taskManagerConfig.timeout(), netConfig);
    +
    +		// computing the amount of memory to use depends on how much memory is available
    +		// it strictly needs to happen AFTER the network stack has been initialized
    +
    +		// check if a value has been configured
    +		long configuredMemory = configuration.getLong(ConfigConstants.TASK_MANAGER_MEMORY_SIZE_KEY, -1L);
    +		checkConfigParameter(configuredMemory == -1 || configuredMemory > 0, configuredMemory,
    +			ConfigConstants.TASK_MANAGER_MEMORY_SIZE_KEY,
    +			"MemoryManager needs at least one MB of memory. " +
    +				"If you leave this config parameter empty, the system automatically " +
    +				"pick a fraction of the available memory.");
    +
    +		long memorySize;
    +		boolean preAllocateMemory = configuration.getBoolean(
    +			ConfigConstants.TASK_MANAGER_MEMORY_PRE_ALLOCATE_KEY,
    +			ConfigConstants.DEFAULT_TASK_MANAGER_MEMORY_PRE_ALLOCATE);
    +		if (configuredMemory > 0) {
    +			if (preAllocateMemory) {
    +				LOG.info("Using {} MB for managed memory." , configuredMemory);
    +			} else {
    +				LOG.info("Limiting managed memory to {} MB, memory will be allocated lazily." , configuredMemory);
    +			}
    +			memorySize = configuredMemory << 20; // megabytes to bytes
    +		} else {
    +			float fraction = configuration.getFloat(
    +				ConfigConstants.TASK_MANAGER_MEMORY_FRACTION_KEY,
    +				ConfigConstants.DEFAULT_MEMORY_MANAGER_MEMORY_FRACTION);
    +			checkConfigParameter(fraction > 0.0f && fraction < 1.0f, fraction,
    +				ConfigConstants.TASK_MANAGER_MEMORY_FRACTION_KEY,
    +				"MemoryManager fraction of the free memory must be between 0.0 and 1.0");
    +
    +			if (memType == MemoryType.HEAP) {
    +				long relativeMemSize = (long) (EnvironmentInformation.getSizeOfFreeHeapMemoryWithDefrag() * fraction);
    +				if (preAllocateMemory) {
    +					LOG.info("Using {} of the currently free heap space for managed heap memory ({} MB)." ,
    +						fraction , relativeMemSize >> 20);
    +				} else {
    +					LOG.info("Limiting managed memory to {} of the currently free heap space ({} MB), " +
    +						"memory will be allocated lazily." , fraction , relativeMemSize >> 20);
    +				}
    +				memorySize = relativeMemSize;
    +			} else if (memType == MemoryType.OFF_HEAP) {
    +				// The maximum heap memory has been adjusted according to the fraction
    +				long maxMemory = EnvironmentInformation.getMaxJvmHeapMemory();
    +				long directMemorySize = (long) (maxMemory / (1.0 - fraction) * fraction);
    +				if (preAllocateMemory) {
    +					LOG.info("Using {} of the maximum memory size for managed off-heap memory ({} MB)." ,
    +						fraction, directMemorySize >> 20);
    +				} else {
    +					LOG.info("Limiting managed memory to {} of the maximum memory size ({} MB)," +
    +						" memory will be allocated lazily.", fraction, directMemorySize >> 20);
    +				}
    +				memorySize = directMemorySize;
    +			} else {
    +				throw new RuntimeException("No supported memory type detected.");
    +			}
    +		}
    +
    +		// now start the memory manager
    +		MemoryManager memoryManager;
    +		try {
    +			memoryManager = new MemoryManager(
    +				memorySize,
    +				taskManagerConfig.numberOfSlots(),
    +				netConfig.networkBufferSize(),
    +				memType,
    +				preAllocateMemory);
    +		} catch (OutOfMemoryError e) {
    +			if (memType == MemoryType.HEAP) {
    +				throw new Exception("OutOfMemory error (" + e.getMessage() +
    +					") while allocating the TaskManager heap memory (" + memorySize + " bytes).", e);
    +			} else if (memType == MemoryType.OFF_HEAP) {
    +				throw new Exception("OutOfMemory error (" + e.getMessage() +
    +					") while allocating the TaskManager off-heap memory (" + memorySize +
    +					" bytes).Try increasing the maximum direct memory (-XX:MaxDirectMemorySize)", e);
    +			} else {
    +				throw e;
    +			}
    +		}
    +
    +		// start the I/O manager, it will create some temp directories.
    +		IOManager ioManager = new IOManagerAsync(taskManagerConfig.tmpDirPaths());
    +
    +		if (leaderRetrievalService == null){
    +			leaderRetrievalService = LeaderRetrievalUtils.createLeaderRetrievalService(configuration);
    +		}
    +
    +		// create the actor properties (which define the actor constructor parameters)
    +		Props tmProps = Props.create(
    +			taskManagerClass,
    +			taskManagerConfig,
    +			resourceID,
    +			connectionInfo,
    +			memoryManager,
    +			ioManager,
    +			network,
    +			taskManagerConfig.numberOfSlots(),
    +			leaderRetrievalService);
    --- End diff --
    
    Also, it doesn't match the constructor.


---
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.
---

[GitHub] flink pull request #2400: [FLINK-4363] Implement TaskManager basic startup o...

Posted by mxm <gi...@git.apache.org>.
Github user mxm commented on a diff in the pull request:

    https://github.com/apache/flink/pull/2400#discussion_r75647172
  
    --- Diff: flink-runtime/src/main/java/org/apache/flink/runtime/rpc/taskexecutor/TaskExecutor.java ---
    @@ -35,27 +79,634 @@
      */
     public class TaskExecutor extends RpcEndpoint<TaskExecutorGateway> {
     
    -	/** The unique resource ID of this TaskExecutor */
    +	private static final Logger LOG = LoggerFactory.getLogger(TaskExecutor.class);
    +
    +	/** Return code for critical errors during the runtime */
    +	private static final int RUNTIME_FAILURE_RETURN_CODE = 2;
    +
    +	/** The name of the TaskManager actor */
    +	private static final String TASK_MANAGER_NAME = "taskmanager";
    +
    +	/** The unique resource ID of this TaskManager */
     	private final ResourceID resourceID;
     
     	/** The access to the leader election and metadata storage services */
     	private final HighAvailabilityServices haServices;
     
    -	// --------- resource manager --------
    +	/** The task manager configuration */
    +	private final TaskManagerConfiguration taskManagerConfig;
    +
    +	/** The connection information of the task manager */
    +	private final InstanceConnectionInfo connectionInfo;
    +
    +	/** The I/O manager component in the task manager */
    +	private final IOManager ioManager;
    +
    +	/** The memory manager component in the task manager */
    +	private final MemoryManager memoryManager;
    +
    +	/** The network component in the task manager */
    +	private final NetworkEnvironment networkEnvironment;
    +
    +	/** The number of slots in the task manager, should be 1 for YARN */
    +	private final int numberOfSlots;
     
    +	// --------- resource manager --------
     	private TaskExecutorToResourceManagerConnection resourceManagerConnection;
     
     	// ------------------------------------------------------------------------
     
     	public TaskExecutor(
    +			TaskManagerConfiguration taskManagerConfig,
    +			ResourceID resourceID,
    +			InstanceConnectionInfo connectionInfo,
    +			MemoryManager memoryManager,
    +			IOManager ioManager,
    +			NetworkEnvironment networkEnvironment,
    +			int numberOfSlots,
     			RpcService rpcService,
    -			HighAvailabilityServices haServices,
    -			ResourceID resourceID) {
    +			HighAvailabilityServices haServices) {
     
     		super(rpcService);
     
    -		this.haServices = checkNotNull(haServices);
    +		this.taskManagerConfig = checkNotNull(taskManagerConfig);
     		this.resourceID = checkNotNull(resourceID);
    +		this.connectionInfo = checkNotNull(connectionInfo);
    +		this.memoryManager = checkNotNull(memoryManager);
    +		this.ioManager = checkNotNull(ioManager);
    +		this.networkEnvironment = checkNotNull(networkEnvironment);
    +		this.numberOfSlots = checkNotNull(numberOfSlots);
    +		this.haServices = checkNotNull(haServices);
    +	}
    +
    +	/**
    +	 * Starts and runs the TaskManager.
    +	 * <p/>
    +	 * This method first tries to select the network interface to use for the TaskManager
    +	 * communication. The network interface is used both for the actor communication
    +	 * (coordination) as well as for the data exchange between task managers. Unless
    +	 * the hostname/interface is explicitly configured in the configuration, this
    +	 * method will try out various interfaces and methods to connect to the JobManager
    +	 * and select the one where the connection attempt is successful.
    +	 * <p/>
    +	 * After selecting the network interface, this method brings up an actor system
    +	 * for the TaskManager and its actors, starts the TaskManager's services
    +	 * (library cache, shuffle network stack, ...), and starts the TaskManager itself.
    +	 *
    +	 * @param configuration    The configuration for the TaskManager.
    +	 * @param taskManagerClass The actor class to instantiate.
    +	 *                         Allows to use TaskManager subclasses for example for YARN.
    +	 */
    +	public static void selectNetworkInterfaceAndRunTaskManager(
    +		Configuration configuration,
    +		ResourceID resourceID,
    +		Class<? extends TaskManager> taskManagerClass) throws Exception {
    +
    +		Tuple2<String, Integer> tuple2 = selectNetworkInterfaceAndPort(configuration);
    +
    +		runTaskManager(tuple2._1(), resourceID, tuple2._2(), configuration, taskManagerClass);
    +	}
    +
    +	private static Tuple2<String, Integer> selectNetworkInterfaceAndPort(Configuration configuration)
    +		throws Exception {
    +		String taskManagerHostname = configuration.getString(ConfigConstants.TASK_MANAGER_HOSTNAME_KEY, null);
    +		if (taskManagerHostname != null) {
    +			LOG.info("Using configured hostname/address for TaskManager: " + taskManagerHostname);
    +		} else {
    +			LeaderRetrievalService leaderRetrievalService = LeaderRetrievalUtils.createLeaderRetrievalService(configuration);
    +			FiniteDuration lookupTimeout = AkkaUtils.getLookupTimeout(configuration);
    +
    +			InetAddress taskManagerAddress = LeaderRetrievalUtils.findConnectingAddress(leaderRetrievalService, lookupTimeout);
    +			taskManagerHostname = taskManagerAddress.getHostName();
    +			LOG.info("TaskManager will use hostname/address '{}' ({}) for communication.",
    +				taskManagerHostname, taskManagerAddress.getHostAddress());
    +		}
    +
    +		// if no task manager port has been configured, use 0 (system will pick any free port)
    +		int actorSystemPort = configuration.getInteger(ConfigConstants.TASK_MANAGER_IPC_PORT_KEY, 0);
    +		if (actorSystemPort < 0 || actorSystemPort > 65535) {
    +			throw new IllegalConfigurationException("Invalid value for '" +
    +				ConfigConstants.TASK_MANAGER_IPC_PORT_KEY +
    +				"' (port for the TaskManager actor system) : " + actorSystemPort +
    +				" - Leave config parameter empty or use 0 to let the system choose a port automatically.");
    +		}
    +
    +		return new Tuple2<>(taskManagerHostname, actorSystemPort);
    +	}
    +
    +	/**
    +	 * Starts and runs the TaskManager. Brings up an actor system for the TaskManager and its
    +	 * actors, starts the TaskManager's services (library cache, shuffle network stack, ...),
    +	 * and starts the TaskManager itself.
    +	 * <p/>
    +	 * This method will also spawn a process reaper for the TaskManager (kill the process if
    +	 * the actor fails) and optionally start the JVM memory logging thread.
    +	 *
    +	 * @param taskManagerHostname The hostname/address of the interface where the actor system
    +	 *                            will communicate.
    +	 * @param resourceID          The id of the resource which the task manager will run on.
    +	 * @param actorSystemPort     The port at which the actor system will communicate.
    +	 * @param configuration       The configuration for the TaskManager.
    +	 * @param taskManagerClass    The actor class to instantiate. Allows the use of TaskManager
    +	 *                            subclasses for example for YARN.
    +	 */
    +	private static void runTaskManager(
    +		String taskManagerHostname,
    +		ResourceID resourceID,
    +		int actorSystemPort,
    +		Configuration configuration,
    +		Class<? extends TaskManager> taskManagerClass) throws Exception {
    +
    +		LOG.info("Starting TaskManager");
    +
    +		// Bring up the TaskManager actor system first, bind it to the given address.
    +
    +		LOG.info("Starting TaskManager actor system at " +
    +			NetUtils.hostAndPortToUrlString(taskManagerHostname, actorSystemPort));
    +
    +		ActorSystem taskManagerSystem;
    +		try {
    +			Tuple2<String, Object> address = new Tuple2<String, Object>(taskManagerHostname, actorSystemPort);
    +			Config akkaConfig = AkkaUtils.getAkkaConfig(configuration, new Some<>(address));
    +			LOG.debug("Using akka configuration\n " + akkaConfig);
    +			taskManagerSystem = AkkaUtils.createActorSystem(akkaConfig);
    +		} catch (Throwable t) {
    +			if (t instanceof org.jboss.netty.channel.ChannelException) {
    +				Throwable cause = t.getCause();
    +				if (cause != null && t.getCause() instanceof java.net.BindException) {
    +					String address = NetUtils.hostAndPortToUrlString(taskManagerHostname, actorSystemPort);
    +					throw new IOException("Unable to bind TaskManager actor system to address " +
    +						address + " - " + cause.getMessage(), t);
    +				}
    +			}
    +			throw new Exception("Could not create TaskManager actor system", t);
    +		}
    +
    +		// start all the TaskManager services (network stack,  library cache, ...)
    +		// and the TaskManager actor
    +		try {
    +			LOG.info("Starting TaskManager actor");
    +			ActorRef taskManagerActor = startTaskManagerComponentsAndActor(
    +				configuration,
    +				resourceID,
    +				taskManagerSystem,
    +				taskManagerHostname,
    +				TASK_MANAGER_NAME,
    +				null,
    +				false,
    +				taskManagerClass);
    +
    +			// start a process reaper that watches the JobManager. If the TaskManager actor dies,
    +			// the process reaper will kill the JVM process (to ensure easy failure detection)
    +			LOG.debug("Starting TaskManager process reaper");
    +
    +			taskManagerSystem.actorOf(
    +				Props.create(ProcessReaper.class, taskManagerActor, LOG, RUNTIME_FAILURE_RETURN_CODE),
    +				"TaskManager_Process_Reaper");
    +
    +			// if desired, start the logging daemon that periodically logs the memory usage information
    +			if (LOG.isInfoEnabled() && configuration.getBoolean(
    +				ConfigConstants.TASK_MANAGER_DEBUG_MEMORY_USAGE_START_LOG_THREAD,
    +				ConfigConstants.DEFAULT_TASK_MANAGER_DEBUG_MEMORY_USAGE_START_LOG_THREAD)) {
    +				LOG.info("Starting periodic memory usage logger");
    +
    +				long interval = configuration.getLong(
    +					ConfigConstants.TASK_MANAGER_DEBUG_MEMORY_USAGE_LOG_INTERVAL_MS,
    +					ConfigConstants.DEFAULT_TASK_MANAGER_DEBUG_MEMORY_USAGE_LOG_INTERVAL_MS);
    +
    +				MemoryLogger logger = new MemoryLogger(LOG, interval, taskManagerSystem);
    +				logger.start();
    +			}
    +
    +			// block until everything is done
    +			taskManagerSystem.awaitTermination();
    +		} catch (Throwable t) {
    +			LOG.error("Error while starting up taskManager", t);
    --- End diff --
    
    This message can be misleading because an exception can also be thrown after startup.


---
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.
---

[GitHub] flink pull request #2400: [FLINK-4363] Implement TaskManager basic startup o...

Posted by wangzhijiang999 <gi...@git.apache.org>.
Github user wangzhijiang999 commented on a diff in the pull request:

    https://github.com/apache/flink/pull/2400#discussion_r75651298
  
    --- Diff: flink-runtime/src/main/java/org/apache/flink/runtime/rpc/taskexecutor/TaskExecutor.java ---
    @@ -35,27 +79,634 @@
      */
     public class TaskExecutor extends RpcEndpoint<TaskExecutorGateway> {
     
    -	/** The unique resource ID of this TaskExecutor */
    +	private static final Logger LOG = LoggerFactory.getLogger(TaskExecutor.class);
    +
    +	/** Return code for critical errors during the runtime */
    +	private static final int RUNTIME_FAILURE_RETURN_CODE = 2;
    +
    +	/** The name of the TaskManager actor */
    +	private static final String TASK_MANAGER_NAME = "taskmanager";
    +
    +	/** The unique resource ID of this TaskManager */
     	private final ResourceID resourceID;
     
     	/** The access to the leader election and metadata storage services */
     	private final HighAvailabilityServices haServices;
     
    -	// --------- resource manager --------
    +	/** The task manager configuration */
    +	private final TaskManagerConfiguration taskManagerConfig;
    +
    +	/** The connection information of the task manager */
    +	private final InstanceConnectionInfo connectionInfo;
    +
    +	/** The I/O manager component in the task manager */
    +	private final IOManager ioManager;
    +
    +	/** The memory manager component in the task manager */
    +	private final MemoryManager memoryManager;
    +
    +	/** The network component in the task manager */
    +	private final NetworkEnvironment networkEnvironment;
    +
    +	/** The number of slots in the task manager, should be 1 for YARN */
    +	private final int numberOfSlots;
     
    +	// --------- resource manager --------
     	private TaskExecutorToResourceManagerConnection resourceManagerConnection;
     
     	// ------------------------------------------------------------------------
     
     	public TaskExecutor(
    +			TaskManagerConfiguration taskManagerConfig,
    +			ResourceID resourceID,
    +			InstanceConnectionInfo connectionInfo,
    +			MemoryManager memoryManager,
    +			IOManager ioManager,
    +			NetworkEnvironment networkEnvironment,
    +			int numberOfSlots,
     			RpcService rpcService,
    -			HighAvailabilityServices haServices,
    -			ResourceID resourceID) {
    +			HighAvailabilityServices haServices) {
     
     		super(rpcService);
     
    -		this.haServices = checkNotNull(haServices);
    +		this.taskManagerConfig = checkNotNull(taskManagerConfig);
     		this.resourceID = checkNotNull(resourceID);
    +		this.connectionInfo = checkNotNull(connectionInfo);
    +		this.memoryManager = checkNotNull(memoryManager);
    +		this.ioManager = checkNotNull(ioManager);
    +		this.networkEnvironment = checkNotNull(networkEnvironment);
    +		this.numberOfSlots = checkNotNull(numberOfSlots);
    +		this.haServices = checkNotNull(haServices);
    +	}
    +
    +	/**
    +	 * Starts and runs the TaskManager.
    +	 * <p/>
    +	 * This method first tries to select the network interface to use for the TaskManager
    +	 * communication. The network interface is used both for the actor communication
    +	 * (coordination) as well as for the data exchange between task managers. Unless
    +	 * the hostname/interface is explicitly configured in the configuration, this
    +	 * method will try out various interfaces and methods to connect to the JobManager
    +	 * and select the one where the connection attempt is successful.
    +	 * <p/>
    +	 * After selecting the network interface, this method brings up an actor system
    +	 * for the TaskManager and its actors, starts the TaskManager's services
    +	 * (library cache, shuffle network stack, ...), and starts the TaskManager itself.
    +	 *
    +	 * @param configuration    The configuration for the TaskManager.
    +	 * @param taskManagerClass The actor class to instantiate.
    +	 *                         Allows to use TaskManager subclasses for example for YARN.
    +	 */
    +	public static void selectNetworkInterfaceAndRunTaskManager(
    +		Configuration configuration,
    +		ResourceID resourceID,
    +		Class<? extends TaskManager> taskManagerClass) throws Exception {
    +
    +		Tuple2<String, Integer> tuple2 = selectNetworkInterfaceAndPort(configuration);
    +
    +		runTaskManager(tuple2._1(), resourceID, tuple2._2(), configuration, taskManagerClass);
    +	}
    +
    +	private static Tuple2<String, Integer> selectNetworkInterfaceAndPort(Configuration configuration)
    +		throws Exception {
    +		String taskManagerHostname = configuration.getString(ConfigConstants.TASK_MANAGER_HOSTNAME_KEY, null);
    +		if (taskManagerHostname != null) {
    +			LOG.info("Using configured hostname/address for TaskManager: " + taskManagerHostname);
    +		} else {
    +			LeaderRetrievalService leaderRetrievalService = LeaderRetrievalUtils.createLeaderRetrievalService(configuration);
    +			FiniteDuration lookupTimeout = AkkaUtils.getLookupTimeout(configuration);
    +
    +			InetAddress taskManagerAddress = LeaderRetrievalUtils.findConnectingAddress(leaderRetrievalService, lookupTimeout);
    +			taskManagerHostname = taskManagerAddress.getHostName();
    +			LOG.info("TaskManager will use hostname/address '{}' ({}) for communication.",
    +				taskManagerHostname, taskManagerAddress.getHostAddress());
    +		}
    +
    +		// if no task manager port has been configured, use 0 (system will pick any free port)
    +		int actorSystemPort = configuration.getInteger(ConfigConstants.TASK_MANAGER_IPC_PORT_KEY, 0);
    +		if (actorSystemPort < 0 || actorSystemPort > 65535) {
    +			throw new IllegalConfigurationException("Invalid value for '" +
    +				ConfigConstants.TASK_MANAGER_IPC_PORT_KEY +
    +				"' (port for the TaskManager actor system) : " + actorSystemPort +
    +				" - Leave config parameter empty or use 0 to let the system choose a port automatically.");
    +		}
    +
    +		return new Tuple2<>(taskManagerHostname, actorSystemPort);
    +	}
    +
    +	/**
    +	 * Starts and runs the TaskManager. Brings up an actor system for the TaskManager and its
    +	 * actors, starts the TaskManager's services (library cache, shuffle network stack, ...),
    +	 * and starts the TaskManager itself.
    +	 * <p/>
    +	 * This method will also spawn a process reaper for the TaskManager (kill the process if
    +	 * the actor fails) and optionally start the JVM memory logging thread.
    +	 *
    +	 * @param taskManagerHostname The hostname/address of the interface where the actor system
    +	 *                            will communicate.
    +	 * @param resourceID          The id of the resource which the task manager will run on.
    +	 * @param actorSystemPort     The port at which the actor system will communicate.
    +	 * @param configuration       The configuration for the TaskManager.
    +	 * @param taskManagerClass    The actor class to instantiate. Allows the use of TaskManager
    +	 *                            subclasses for example for YARN.
    +	 */
    +	private static void runTaskManager(
    +		String taskManagerHostname,
    +		ResourceID resourceID,
    +		int actorSystemPort,
    +		Configuration configuration,
    +		Class<? extends TaskManager> taskManagerClass) throws Exception {
    +
    +		LOG.info("Starting TaskManager");
    +
    +		// Bring up the TaskManager actor system first, bind it to the given address.
    +
    +		LOG.info("Starting TaskManager actor system at " +
    +			NetUtils.hostAndPortToUrlString(taskManagerHostname, actorSystemPort));
    +
    +		ActorSystem taskManagerSystem;
    +		try {
    +			Tuple2<String, Object> address = new Tuple2<String, Object>(taskManagerHostname, actorSystemPort);
    +			Config akkaConfig = AkkaUtils.getAkkaConfig(configuration, new Some<>(address));
    +			LOG.debug("Using akka configuration\n " + akkaConfig);
    +			taskManagerSystem = AkkaUtils.createActorSystem(akkaConfig);
    +		} catch (Throwable t) {
    +			if (t instanceof org.jboss.netty.channel.ChannelException) {
    +				Throwable cause = t.getCause();
    +				if (cause != null && t.getCause() instanceof java.net.BindException) {
    +					String address = NetUtils.hostAndPortToUrlString(taskManagerHostname, actorSystemPort);
    +					throw new IOException("Unable to bind TaskManager actor system to address " +
    +						address + " - " + cause.getMessage(), t);
    +				}
    +			}
    +			throw new Exception("Could not create TaskManager actor system", t);
    +		}
    +
    +		// start all the TaskManager services (network stack,  library cache, ...)
    +		// and the TaskManager actor
    +		try {
    +			LOG.info("Starting TaskManager actor");
    +			ActorRef taskManagerActor = startTaskManagerComponentsAndActor(
    +				configuration,
    +				resourceID,
    +				taskManagerSystem,
    +				taskManagerHostname,
    +				TASK_MANAGER_NAME,
    +				null,
    +				false,
    +				taskManagerClass);
    +
    +			// start a process reaper that watches the JobManager. If the TaskManager actor dies,
    +			// the process reaper will kill the JVM process (to ensure easy failure detection)
    +			LOG.debug("Starting TaskManager process reaper");
    +
    +			taskManagerSystem.actorOf(
    +				Props.create(ProcessReaper.class, taskManagerActor, LOG, RUNTIME_FAILURE_RETURN_CODE),
    +				"TaskManager_Process_Reaper");
    +
    +			// if desired, start the logging daemon that periodically logs the memory usage information
    +			if (LOG.isInfoEnabled() && configuration.getBoolean(
    +				ConfigConstants.TASK_MANAGER_DEBUG_MEMORY_USAGE_START_LOG_THREAD,
    +				ConfigConstants.DEFAULT_TASK_MANAGER_DEBUG_MEMORY_USAGE_START_LOG_THREAD)) {
    +				LOG.info("Starting periodic memory usage logger");
    +
    +				long interval = configuration.getLong(
    +					ConfigConstants.TASK_MANAGER_DEBUG_MEMORY_USAGE_LOG_INTERVAL_MS,
    +					ConfigConstants.DEFAULT_TASK_MANAGER_DEBUG_MEMORY_USAGE_LOG_INTERVAL_MS);
    +
    +				MemoryLogger logger = new MemoryLogger(LOG, interval, taskManagerSystem);
    +				logger.start();
    +			}
    +
    +			// block until everything is done
    +			taskManagerSystem.awaitTermination();
    +		} catch (Throwable t) {
    +			LOG.error("Error while starting up taskManager", t);
    +			try {
    +				taskManagerSystem.shutdown();
    +			} catch (Throwable tt) {
    +				LOG.warn("Could not cleanly shut down actor system", tt);
    +			}
    +			throw t;
    +		}
    +	}
    +
    +	// --------------------------------------------------------------------------
    +	//  Starting and running the TaskManager
    +	// --------------------------------------------------------------------------
    +
    +	/**
    +	 * @param configuration                 The configuration for the TaskManager.
    +	 * @param resourceID                    The id of the resource which the task manager will run on.
    +	 * @param actorSystem                  The actor system that should run the TaskManager actor.
    +	 * @param taskManagerHostname       The hostname/address that describes the TaskManager's data location.
    +	 * @param taskManagerActorName      Optionally the name of the TaskManager actor. If none is given,
    +	 *                                      the actor will use a random name.
    +	 * @param leaderRetrievalService        Optionally, a leader retrieval service can be provided. If none is given,
    +	 *                                      then a LeaderRetrievalService is constructed from the configuration.
    +	 * @param localTaskManagerCommunication If true, the TaskManager will not initiate the TCP network stack.
    +	 * @param taskManagerClass      The class of the TaskManager actor. May be used to give
    +	 *                                      subclasses that understand additional actor messages.
    +	 * @return An ActorRef to the TaskManager actor.
    +	 * @throws org.apache.flink.configuration.IllegalConfigurationException     Thrown, if the given config contains illegal values.
    +	 * @throws java.io.IOException      Thrown, if any of the I/O components (such as buffer pools,
    +	 *                                       I/O manager, ...) cannot be properly started.
    +	 * @throws java.lang.Exception      Thrown is some other error occurs while parsing the configuration
    +	 *                                      or starting the TaskManager components.
    +	 */
    +	public static ActorRef startTaskManagerComponentsAndActor(
    +		Configuration configuration,
    +		ResourceID resourceID,
    +		ActorSystem actorSystem,
    +		String taskManagerHostname,
    +		String taskManagerActorName,
    +		LeaderRetrievalService leaderRetrievalService,
    +		boolean localTaskManagerCommunication,
    +		Class<? extends TaskManager> taskManagerClass) throws Exception {
    +
    +		Tuple4<TaskManagerConfiguration, NetworkEnvironmentConfiguration, InstanceConnectionInfo, MemoryType> tuple4
    +			= parseTaskManagerConfiguration(configuration, taskManagerHostname, localTaskManagerCommunication);
    +
    +		TaskManagerConfiguration taskManagerConfig = tuple4._1();
    +		NetworkEnvironmentConfiguration netConfig = tuple4._2();
    +		InstanceConnectionInfo connectionInfo = tuple4._3();
    +		MemoryType memType = tuple4._4();
    +
    +		// pre-start checks
    +		checkTempDirs(taskManagerConfig.tmpDirPaths());
    +
    +		ExecutionContext executionContext = ExecutionContexts$.MODULE$.fromExecutor(new ForkJoinPool());
    +
    +		// we start the network first, to make sure it can allocate its buffers first
    +		NetworkEnvironment network = new NetworkEnvironment(executionContext, taskManagerConfig.timeout(), netConfig);
    +
    +		// computing the amount of memory to use depends on how much memory is available
    +		// it strictly needs to happen AFTER the network stack has been initialized
    +
    +		// check if a value has been configured
    +		long configuredMemory = configuration.getLong(ConfigConstants.TASK_MANAGER_MEMORY_SIZE_KEY, -1L);
    +		checkConfigParameter(configuredMemory == -1 || configuredMemory > 0, configuredMemory,
    +			ConfigConstants.TASK_MANAGER_MEMORY_SIZE_KEY,
    +			"MemoryManager needs at least one MB of memory. " +
    +				"If you leave this config parameter empty, the system automatically " +
    +				"pick a fraction of the available memory.");
    +
    +		long memorySize;
    +		boolean preAllocateMemory = configuration.getBoolean(
    +			ConfigConstants.TASK_MANAGER_MEMORY_PRE_ALLOCATE_KEY,
    +			ConfigConstants.DEFAULT_TASK_MANAGER_MEMORY_PRE_ALLOCATE);
    +		if (configuredMemory > 0) {
    +			if (preAllocateMemory) {
    +				LOG.info("Using {} MB for managed memory." , configuredMemory);
    +			} else {
    +				LOG.info("Limiting managed memory to {} MB, memory will be allocated lazily." , configuredMemory);
    +			}
    +			memorySize = configuredMemory << 20; // megabytes to bytes
    +		} else {
    +			float fraction = configuration.getFloat(
    +				ConfigConstants.TASK_MANAGER_MEMORY_FRACTION_KEY,
    +				ConfigConstants.DEFAULT_MEMORY_MANAGER_MEMORY_FRACTION);
    +			checkConfigParameter(fraction > 0.0f && fraction < 1.0f, fraction,
    +				ConfigConstants.TASK_MANAGER_MEMORY_FRACTION_KEY,
    +				"MemoryManager fraction of the free memory must be between 0.0 and 1.0");
    +
    +			if (memType == MemoryType.HEAP) {
    +				long relativeMemSize = (long) (EnvironmentInformation.getSizeOfFreeHeapMemoryWithDefrag() * fraction);
    +				if (preAllocateMemory) {
    +					LOG.info("Using {} of the currently free heap space for managed heap memory ({} MB)." ,
    +						fraction , relativeMemSize >> 20);
    +				} else {
    +					LOG.info("Limiting managed memory to {} of the currently free heap space ({} MB), " +
    +						"memory will be allocated lazily." , fraction , relativeMemSize >> 20);
    +				}
    +				memorySize = relativeMemSize;
    +			} else if (memType == MemoryType.OFF_HEAP) {
    +				// The maximum heap memory has been adjusted according to the fraction
    +				long maxMemory = EnvironmentInformation.getMaxJvmHeapMemory();
    +				long directMemorySize = (long) (maxMemory / (1.0 - fraction) * fraction);
    +				if (preAllocateMemory) {
    +					LOG.info("Using {} of the maximum memory size for managed off-heap memory ({} MB)." ,
    +						fraction, directMemorySize >> 20);
    +				} else {
    +					LOG.info("Limiting managed memory to {} of the maximum memory size ({} MB)," +
    +						" memory will be allocated lazily.", fraction, directMemorySize >> 20);
    +				}
    +				memorySize = directMemorySize;
    +			} else {
    +				throw new RuntimeException("No supported memory type detected.");
    +			}
    +		}
    +
    +		// now start the memory manager
    +		MemoryManager memoryManager;
    +		try {
    +			memoryManager = new MemoryManager(
    +				memorySize,
    +				taskManagerConfig.numberOfSlots(),
    +				netConfig.networkBufferSize(),
    +				memType,
    +				preAllocateMemory);
    +		} catch (OutOfMemoryError e) {
    +			if (memType == MemoryType.HEAP) {
    +				throw new Exception("OutOfMemory error (" + e.getMessage() +
    +					") while allocating the TaskManager heap memory (" + memorySize + " bytes).", e);
    +			} else if (memType == MemoryType.OFF_HEAP) {
    +				throw new Exception("OutOfMemory error (" + e.getMessage() +
    +					") while allocating the TaskManager off-heap memory (" + memorySize +
    +					" bytes).Try increasing the maximum direct memory (-XX:MaxDirectMemorySize)", e);
    +			} else {
    +				throw e;
    +			}
    +		}
    +
    +		// start the I/O manager, it will create some temp directories.
    +		IOManager ioManager = new IOManagerAsync(taskManagerConfig.tmpDirPaths());
    +
    +		if (leaderRetrievalService == null){
    +			leaderRetrievalService = LeaderRetrievalUtils.createLeaderRetrievalService(configuration);
    --- End diff --
    
    It just follows the previous implementation of TM currently. In the new constructor, the leader retrieval service should be replaced by 'HighAvailabilityServices'. So the related parts of LeaderRetrievalService and ActorSystem should be fixed later.


---
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.
---

[GitHub] flink pull request #2400: [FLINK-4363] Implement TaskManager basic startup o...

Posted by wangzhijiang999 <gi...@git.apache.org>.
Github user wangzhijiang999 commented on a diff in the pull request:

    https://github.com/apache/flink/pull/2400#discussion_r75649383
  
    --- Diff: flink-runtime/src/main/java/org/apache/flink/runtime/rpc/taskexecutor/TaskExecutor.java ---
    @@ -35,27 +79,634 @@
      */
     public class TaskExecutor extends RpcEndpoint<TaskExecutorGateway> {
     
    -	/** The unique resource ID of this TaskExecutor */
    +	private static final Logger LOG = LoggerFactory.getLogger(TaskExecutor.class);
    +
    +	/** Return code for critical errors during the runtime */
    +	private static final int RUNTIME_FAILURE_RETURN_CODE = 2;
    +
    +	/** The name of the TaskManager actor */
    +	private static final String TASK_MANAGER_NAME = "taskmanager";
    +
    +	/** The unique resource ID of this TaskManager */
     	private final ResourceID resourceID;
     
     	/** The access to the leader election and metadata storage services */
     	private final HighAvailabilityServices haServices;
     
    -	// --------- resource manager --------
    +	/** The task manager configuration */
    +	private final TaskManagerConfiguration taskManagerConfig;
    +
    +	/** The connection information of the task manager */
    +	private final InstanceConnectionInfo connectionInfo;
    +
    +	/** The I/O manager component in the task manager */
    +	private final IOManager ioManager;
    +
    +	/** The memory manager component in the task manager */
    +	private final MemoryManager memoryManager;
    +
    +	/** The network component in the task manager */
    +	private final NetworkEnvironment networkEnvironment;
    +
    +	/** The number of slots in the task manager, should be 1 for YARN */
    +	private final int numberOfSlots;
     
    +	// --------- resource manager --------
     	private TaskExecutorToResourceManagerConnection resourceManagerConnection;
     
     	// ------------------------------------------------------------------------
     
     	public TaskExecutor(
    +			TaskManagerConfiguration taskManagerConfig,
    +			ResourceID resourceID,
    +			InstanceConnectionInfo connectionInfo,
    +			MemoryManager memoryManager,
    +			IOManager ioManager,
    +			NetworkEnvironment networkEnvironment,
    +			int numberOfSlots,
     			RpcService rpcService,
    -			HighAvailabilityServices haServices,
    -			ResourceID resourceID) {
    +			HighAvailabilityServices haServices) {
     
     		super(rpcService);
     
    -		this.haServices = checkNotNull(haServices);
    +		this.taskManagerConfig = checkNotNull(taskManagerConfig);
     		this.resourceID = checkNotNull(resourceID);
    +		this.connectionInfo = checkNotNull(connectionInfo);
    +		this.memoryManager = checkNotNull(memoryManager);
    +		this.ioManager = checkNotNull(ioManager);
    +		this.networkEnvironment = checkNotNull(networkEnvironment);
    +		this.numberOfSlots = checkNotNull(numberOfSlots);
    +		this.haServices = checkNotNull(haServices);
    +	}
    +
    +	/**
    +	 * Starts and runs the TaskManager.
    +	 * <p/>
    +	 * This method first tries to select the network interface to use for the TaskManager
    +	 * communication. The network interface is used both for the actor communication
    +	 * (coordination) as well as for the data exchange between task managers. Unless
    +	 * the hostname/interface is explicitly configured in the configuration, this
    +	 * method will try out various interfaces and methods to connect to the JobManager
    +	 * and select the one where the connection attempt is successful.
    +	 * <p/>
    +	 * After selecting the network interface, this method brings up an actor system
    +	 * for the TaskManager and its actors, starts the TaskManager's services
    +	 * (library cache, shuffle network stack, ...), and starts the TaskManager itself.
    +	 *
    +	 * @param configuration    The configuration for the TaskManager.
    +	 * @param taskManagerClass The actor class to instantiate.
    +	 *                         Allows to use TaskManager subclasses for example for YARN.
    +	 */
    +	public static void selectNetworkInterfaceAndRunTaskManager(
    +		Configuration configuration,
    +		ResourceID resourceID,
    +		Class<? extends TaskManager> taskManagerClass) throws Exception {
    +
    +		Tuple2<String, Integer> tuple2 = selectNetworkInterfaceAndPort(configuration);
    +
    +		runTaskManager(tuple2._1(), resourceID, tuple2._2(), configuration, taskManagerClass);
    +	}
    +
    +	private static Tuple2<String, Integer> selectNetworkInterfaceAndPort(Configuration configuration)
    --- End diff --
    
    It is better to use InetAddress instead of scala tupelo


---
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.
---

[GitHub] flink pull request #2400: [FLINK-4363] Implement TaskManager basic startup o...

Posted by wangzhijiang999 <gi...@git.apache.org>.
Github user wangzhijiang999 commented on a diff in the pull request:

    https://github.com/apache/flink/pull/2400#discussion_r75648894
  
    --- Diff: flink-runtime/src/main/java/org/apache/flink/runtime/rpc/taskexecutor/TaskExecutor.java ---
    @@ -35,27 +79,634 @@
      */
     public class TaskExecutor extends RpcEndpoint<TaskExecutorGateway> {
     
    -	/** The unique resource ID of this TaskExecutor */
    +	private static final Logger LOG = LoggerFactory.getLogger(TaskExecutor.class);
    +
    +	/** Return code for critical errors during the runtime */
    +	private static final int RUNTIME_FAILURE_RETURN_CODE = 2;
    +
    +	/** The name of the TaskManager actor */
    +	private static final String TASK_MANAGER_NAME = "taskmanager";
    +
    +	/** The unique resource ID of this TaskManager */
     	private final ResourceID resourceID;
     
     	/** The access to the leader election and metadata storage services */
     	private final HighAvailabilityServices haServices;
     
    -	// --------- resource manager --------
    +	/** The task manager configuration */
    +	private final TaskManagerConfiguration taskManagerConfig;
    +
    +	/** The connection information of the task manager */
    +	private final InstanceConnectionInfo connectionInfo;
    +
    +	/** The I/O manager component in the task manager */
    +	private final IOManager ioManager;
    +
    +	/** The memory manager component in the task manager */
    +	private final MemoryManager memoryManager;
    +
    +	/** The network component in the task manager */
    +	private final NetworkEnvironment networkEnvironment;
    +
    +	/** The number of slots in the task manager, should be 1 for YARN */
    +	private final int numberOfSlots;
     
    +	// --------- resource manager --------
     	private TaskExecutorToResourceManagerConnection resourceManagerConnection;
     
     	// ------------------------------------------------------------------------
     
     	public TaskExecutor(
    +			TaskManagerConfiguration taskManagerConfig,
    +			ResourceID resourceID,
    +			InstanceConnectionInfo connectionInfo,
    +			MemoryManager memoryManager,
    +			IOManager ioManager,
    +			NetworkEnvironment networkEnvironment,
    +			int numberOfSlots,
     			RpcService rpcService,
    -			HighAvailabilityServices haServices,
    -			ResourceID resourceID) {
    +			HighAvailabilityServices haServices) {
     
     		super(rpcService);
     
    -		this.haServices = checkNotNull(haServices);
    +		this.taskManagerConfig = checkNotNull(taskManagerConfig);
     		this.resourceID = checkNotNull(resourceID);
    +		this.connectionInfo = checkNotNull(connectionInfo);
    +		this.memoryManager = checkNotNull(memoryManager);
    +		this.ioManager = checkNotNull(ioManager);
    +		this.networkEnvironment = checkNotNull(networkEnvironment);
    +		this.numberOfSlots = checkNotNull(numberOfSlots);
    +		this.haServices = checkNotNull(haServices);
    +	}
    +
    +	/**
    +	 * Starts and runs the TaskManager.
    +	 * <p/>
    +	 * This method first tries to select the network interface to use for the TaskManager
    +	 * communication. The network interface is used both for the actor communication
    +	 * (coordination) as well as for the data exchange between task managers. Unless
    +	 * the hostname/interface is explicitly configured in the configuration, this
    +	 * method will try out various interfaces and methods to connect to the JobManager
    +	 * and select the one where the connection attempt is successful.
    +	 * <p/>
    +	 * After selecting the network interface, this method brings up an actor system
    +	 * for the TaskManager and its actors, starts the TaskManager's services
    +	 * (library cache, shuffle network stack, ...), and starts the TaskManager itself.
    +	 *
    +	 * @param configuration    The configuration for the TaskManager.
    +	 * @param taskManagerClass The actor class to instantiate.
    +	 *                         Allows to use TaskManager subclasses for example for YARN.
    +	 */
    +	public static void selectNetworkInterfaceAndRunTaskManager(
    +		Configuration configuration,
    +		ResourceID resourceID,
    +		Class<? extends TaskManager> taskManagerClass) throws Exception {
    +
    +		Tuple2<String, Integer> tuple2 = selectNetworkInterfaceAndPort(configuration);
    +
    +		runTaskManager(tuple2._1(), resourceID, tuple2._2(), configuration, taskManagerClass);
    +	}
    +
    +	private static Tuple2<String, Integer> selectNetworkInterfaceAndPort(Configuration configuration)
    +		throws Exception {
    +		String taskManagerHostname = configuration.getString(ConfigConstants.TASK_MANAGER_HOSTNAME_KEY, null);
    +		if (taskManagerHostname != null) {
    +			LOG.info("Using configured hostname/address for TaskManager: " + taskManagerHostname);
    +		} else {
    +			LeaderRetrievalService leaderRetrievalService = LeaderRetrievalUtils.createLeaderRetrievalService(configuration);
    +			FiniteDuration lookupTimeout = AkkaUtils.getLookupTimeout(configuration);
    +
    +			InetAddress taskManagerAddress = LeaderRetrievalUtils.findConnectingAddress(leaderRetrievalService, lookupTimeout);
    +			taskManagerHostname = taskManagerAddress.getHostName();
    +			LOG.info("TaskManager will use hostname/address '{}' ({}) for communication.",
    +				taskManagerHostname, taskManagerAddress.getHostAddress());
    +		}
    +
    +		// if no task manager port has been configured, use 0 (system will pick any free port)
    +		int actorSystemPort = configuration.getInteger(ConfigConstants.TASK_MANAGER_IPC_PORT_KEY, 0);
    +		if (actorSystemPort < 0 || actorSystemPort > 65535) {
    +			throw new IllegalConfigurationException("Invalid value for '" +
    +				ConfigConstants.TASK_MANAGER_IPC_PORT_KEY +
    +				"' (port for the TaskManager actor system) : " + actorSystemPort +
    +				" - Leave config parameter empty or use 0 to let the system choose a port automatically.");
    +		}
    +
    +		return new Tuple2<>(taskManagerHostname, actorSystemPort);
    +	}
    +
    +	/**
    +	 * Starts and runs the TaskManager. Brings up an actor system for the TaskManager and its
    +	 * actors, starts the TaskManager's services (library cache, shuffle network stack, ...),
    +	 * and starts the TaskManager itself.
    +	 * <p/>
    +	 * This method will also spawn a process reaper for the TaskManager (kill the process if
    +	 * the actor fails) and optionally start the JVM memory logging thread.
    +	 *
    +	 * @param taskManagerHostname The hostname/address of the interface where the actor system
    +	 *                            will communicate.
    +	 * @param resourceID          The id of the resource which the task manager will run on.
    +	 * @param actorSystemPort     The port at which the actor system will communicate.
    +	 * @param configuration       The configuration for the TaskManager.
    +	 * @param taskManagerClass    The actor class to instantiate. Allows the use of TaskManager
    +	 *                            subclasses for example for YARN.
    +	 */
    +	private static void runTaskManager(
    +		String taskManagerHostname,
    +		ResourceID resourceID,
    +		int actorSystemPort,
    +		Configuration configuration,
    +		Class<? extends TaskManager> taskManagerClass) throws Exception {
    +
    +		LOG.info("Starting TaskManager");
    +
    +		// Bring up the TaskManager actor system first, bind it to the given address.
    +
    +		LOG.info("Starting TaskManager actor system at " +
    +			NetUtils.hostAndPortToUrlString(taskManagerHostname, actorSystemPort));
    +
    +		ActorSystem taskManagerSystem;
    +		try {
    +			Tuple2<String, Object> address = new Tuple2<String, Object>(taskManagerHostname, actorSystemPort);
    +			Config akkaConfig = AkkaUtils.getAkkaConfig(configuration, new Some<>(address));
    +			LOG.debug("Using akka configuration\n " + akkaConfig);
    +			taskManagerSystem = AkkaUtils.createActorSystem(akkaConfig);
    +		} catch (Throwable t) {
    +			if (t instanceof org.jboss.netty.channel.ChannelException) {
    +				Throwable cause = t.getCause();
    +				if (cause != null && t.getCause() instanceof java.net.BindException) {
    +					String address = NetUtils.hostAndPortToUrlString(taskManagerHostname, actorSystemPort);
    +					throw new IOException("Unable to bind TaskManager actor system to address " +
    +						address + " - " + cause.getMessage(), t);
    +				}
    +			}
    +			throw new Exception("Could not create TaskManager actor system", t);
    +		}
    +
    +		// start all the TaskManager services (network stack,  library cache, ...)
    +		// and the TaskManager actor
    +		try {
    +			LOG.info("Starting TaskManager actor");
    +			ActorRef taskManagerActor = startTaskManagerComponentsAndActor(
    +				configuration,
    +				resourceID,
    +				taskManagerSystem,
    +				taskManagerHostname,
    +				TASK_MANAGER_NAME,
    +				null,
    +				false,
    +				taskManagerClass);
    +
    +			// start a process reaper that watches the JobManager. If the TaskManager actor dies,
    +			// the process reaper will kill the JVM process (to ensure easy failure detection)
    +			LOG.debug("Starting TaskManager process reaper");
    +
    +			taskManagerSystem.actorOf(
    +				Props.create(ProcessReaper.class, taskManagerActor, LOG, RUNTIME_FAILURE_RETURN_CODE),
    +				"TaskManager_Process_Reaper");
    +
    +			// if desired, start the logging daemon that periodically logs the memory usage information
    +			if (LOG.isInfoEnabled() && configuration.getBoolean(
    +				ConfigConstants.TASK_MANAGER_DEBUG_MEMORY_USAGE_START_LOG_THREAD,
    +				ConfigConstants.DEFAULT_TASK_MANAGER_DEBUG_MEMORY_USAGE_START_LOG_THREAD)) {
    +				LOG.info("Starting periodic memory usage logger");
    +
    +				long interval = configuration.getLong(
    +					ConfigConstants.TASK_MANAGER_DEBUG_MEMORY_USAGE_LOG_INTERVAL_MS,
    +					ConfigConstants.DEFAULT_TASK_MANAGER_DEBUG_MEMORY_USAGE_LOG_INTERVAL_MS);
    +
    +				MemoryLogger logger = new MemoryLogger(LOG, interval, taskManagerSystem);
    +				logger.start();
    +			}
    +
    +			// block until everything is done
    +			taskManagerSystem.awaitTermination();
    +		} catch (Throwable t) {
    +			LOG.error("Error while starting up taskManager", t);
    +			try {
    +				taskManagerSystem.shutdown();
    +			} catch (Throwable tt) {
    +				LOG.warn("Could not cleanly shut down actor system", tt);
    +			}
    +			throw t;
    +		}
    +	}
    +
    +	// --------------------------------------------------------------------------
    +	//  Starting and running the TaskManager
    +	// --------------------------------------------------------------------------
    +
    +	/**
    +	 * @param configuration                 The configuration for the TaskManager.
    +	 * @param resourceID                    The id of the resource which the task manager will run on.
    +	 * @param actorSystem                  The actor system that should run the TaskManager actor.
    +	 * @param taskManagerHostname       The hostname/address that describes the TaskManager's data location.
    +	 * @param taskManagerActorName      Optionally the name of the TaskManager actor. If none is given,
    +	 *                                      the actor will use a random name.
    +	 * @param leaderRetrievalService        Optionally, a leader retrieval service can be provided. If none is given,
    +	 *                                      then a LeaderRetrievalService is constructed from the configuration.
    +	 * @param localTaskManagerCommunication If true, the TaskManager will not initiate the TCP network stack.
    +	 * @param taskManagerClass      The class of the TaskManager actor. May be used to give
    +	 *                                      subclasses that understand additional actor messages.
    +	 * @return An ActorRef to the TaskManager actor.
    +	 * @throws org.apache.flink.configuration.IllegalConfigurationException     Thrown, if the given config contains illegal values.
    +	 * @throws java.io.IOException      Thrown, if any of the I/O components (such as buffer pools,
    +	 *                                       I/O manager, ...) cannot be properly started.
    +	 * @throws java.lang.Exception      Thrown is some other error occurs while parsing the configuration
    +	 *                                      or starting the TaskManager components.
    +	 */
    +	public static ActorRef startTaskManagerComponentsAndActor(
    +		Configuration configuration,
    +		ResourceID resourceID,
    +		ActorSystem actorSystem,
    +		String taskManagerHostname,
    +		String taskManagerActorName,
    +		LeaderRetrievalService leaderRetrievalService,
    +		boolean localTaskManagerCommunication,
    +		Class<? extends TaskManager> taskManagerClass) throws Exception {
    +
    +		Tuple4<TaskManagerConfiguration, NetworkEnvironmentConfiguration, InstanceConnectionInfo, MemoryType> tuple4
    +			= parseTaskManagerConfiguration(configuration, taskManagerHostname, localTaskManagerCommunication);
    +
    +		TaskManagerConfiguration taskManagerConfig = tuple4._1();
    +		NetworkEnvironmentConfiguration netConfig = tuple4._2();
    +		InstanceConnectionInfo connectionInfo = tuple4._3();
    +		MemoryType memType = tuple4._4();
    +
    +		// pre-start checks
    +		checkTempDirs(taskManagerConfig.tmpDirPaths());
    +
    +		ExecutionContext executionContext = ExecutionContexts$.MODULE$.fromExecutor(new ForkJoinPool());
    +
    +		// we start the network first, to make sure it can allocate its buffers first
    +		NetworkEnvironment network = new NetworkEnvironment(executionContext, taskManagerConfig.timeout(), netConfig);
    +
    +		// computing the amount of memory to use depends on how much memory is available
    +		// it strictly needs to happen AFTER the network stack has been initialized
    +
    +		// check if a value has been configured
    +		long configuredMemory = configuration.getLong(ConfigConstants.TASK_MANAGER_MEMORY_SIZE_KEY, -1L);
    +		checkConfigParameter(configuredMemory == -1 || configuredMemory > 0, configuredMemory,
    +			ConfigConstants.TASK_MANAGER_MEMORY_SIZE_KEY,
    +			"MemoryManager needs at least one MB of memory. " +
    +				"If you leave this config parameter empty, the system automatically " +
    +				"pick a fraction of the available memory.");
    +
    +		long memorySize;
    +		boolean preAllocateMemory = configuration.getBoolean(
    +			ConfigConstants.TASK_MANAGER_MEMORY_PRE_ALLOCATE_KEY,
    +			ConfigConstants.DEFAULT_TASK_MANAGER_MEMORY_PRE_ALLOCATE);
    +		if (configuredMemory > 0) {
    +			if (preAllocateMemory) {
    +				LOG.info("Using {} MB for managed memory." , configuredMemory);
    +			} else {
    +				LOG.info("Limiting managed memory to {} MB, memory will be allocated lazily." , configuredMemory);
    +			}
    +			memorySize = configuredMemory << 20; // megabytes to bytes
    +		} else {
    +			float fraction = configuration.getFloat(
    +				ConfigConstants.TASK_MANAGER_MEMORY_FRACTION_KEY,
    +				ConfigConstants.DEFAULT_MEMORY_MANAGER_MEMORY_FRACTION);
    +			checkConfigParameter(fraction > 0.0f && fraction < 1.0f, fraction,
    +				ConfigConstants.TASK_MANAGER_MEMORY_FRACTION_KEY,
    +				"MemoryManager fraction of the free memory must be between 0.0 and 1.0");
    +
    +			if (memType == MemoryType.HEAP) {
    +				long relativeMemSize = (long) (EnvironmentInformation.getSizeOfFreeHeapMemoryWithDefrag() * fraction);
    +				if (preAllocateMemory) {
    +					LOG.info("Using {} of the currently free heap space for managed heap memory ({} MB)." ,
    +						fraction , relativeMemSize >> 20);
    +				} else {
    +					LOG.info("Limiting managed memory to {} of the currently free heap space ({} MB), " +
    +						"memory will be allocated lazily." , fraction , relativeMemSize >> 20);
    +				}
    +				memorySize = relativeMemSize;
    +			} else if (memType == MemoryType.OFF_HEAP) {
    +				// The maximum heap memory has been adjusted according to the fraction
    +				long maxMemory = EnvironmentInformation.getMaxJvmHeapMemory();
    +				long directMemorySize = (long) (maxMemory / (1.0 - fraction) * fraction);
    +				if (preAllocateMemory) {
    +					LOG.info("Using {} of the maximum memory size for managed off-heap memory ({} MB)." ,
    +						fraction, directMemorySize >> 20);
    +				} else {
    +					LOG.info("Limiting managed memory to {} of the maximum memory size ({} MB)," +
    +						" memory will be allocated lazily.", fraction, directMemorySize >> 20);
    +				}
    +				memorySize = directMemorySize;
    +			} else {
    +				throw new RuntimeException("No supported memory type detected.");
    +			}
    +		}
    +
    +		// now start the memory manager
    +		MemoryManager memoryManager;
    +		try {
    +			memoryManager = new MemoryManager(
    +				memorySize,
    +				taskManagerConfig.numberOfSlots(),
    +				netConfig.networkBufferSize(),
    +				memType,
    +				preAllocateMemory);
    +		} catch (OutOfMemoryError e) {
    +			if (memType == MemoryType.HEAP) {
    +				throw new Exception("OutOfMemory error (" + e.getMessage() +
    +					") while allocating the TaskManager heap memory (" + memorySize + " bytes).", e);
    +			} else if (memType == MemoryType.OFF_HEAP) {
    +				throw new Exception("OutOfMemory error (" + e.getMessage() +
    +					") while allocating the TaskManager off-heap memory (" + memorySize +
    +					" bytes).Try increasing the maximum direct memory (-XX:MaxDirectMemorySize)", e);
    +			} else {
    +				throw e;
    +			}
    +		}
    +
    +		// start the I/O manager, it will create some temp directories.
    +		IOManager ioManager = new IOManagerAsync(taskManagerConfig.tmpDirPaths());
    +
    +		if (leaderRetrievalService == null){
    +			leaderRetrievalService = LeaderRetrievalUtils.createLeaderRetrievalService(configuration);
    +		}
    +
    +		// create the actor properties (which define the actor constructor parameters)
    +		Props tmProps = Props.create(
    +			taskManagerClass,
    +			taskManagerConfig,
    +			resourceID,
    +			connectionInfo,
    +			memoryManager,
    +			ioManager,
    +			network,
    +			taskManagerConfig.numberOfSlots(),
    +			leaderRetrievalService);
    +
    +		ActorRef taskManagerActorRef;
    +		if (taskManagerActorName != null && !taskManagerActorName.equals("")) {
    +			taskManagerActorRef = actorSystem.actorOf(tmProps, taskManagerActorName);
    +		} else {
    +			taskManagerActorRef = actorSystem.actorOf(tmProps);
    +		}
    +
    +		return taskManagerActorRef;
    +	}
    +
    +	// --------------------------------------------------------------------------
    +	//  Parsing and checking the TaskManager Configuration
    +	// --------------------------------------------------------------------------
    +
    +	/**
    +	 * Utility method to extract TaskManager config parameters from the configuration and to
    +	 * sanity check them.
    +	 *
    +	 * @param configuration                 The configuration.
    +	 * @param taskManagerHostname           The host name under which the TaskManager communicates.
    +	 * @param localTaskManagerCommunication True, to skip initializing the network stack.
    +	 *                                      Use only in cases where only one task manager runs.
    +	 * @return A tuple (TaskManagerConfiguration, network configuration,
    +	 * InstanceConnectionInfo, JobManager actor Akka URL).
    +	 */
    +	private static Tuple4<TaskManagerConfiguration, NetworkEnvironmentConfiguration, InstanceConnectionInfo, MemoryType>
    +		parseTaskManagerConfiguration(Configuration configuration, String taskManagerHostname, boolean localTaskManagerCommunication)
    +		throws Exception {
    +
    +		// ------- read values from the config and check them ---------
    +		//                      (a lot of them)
    +
    +		// ----> hosts / ports for communication and data exchange
    +
    +		int dataport = configuration.getInteger(ConfigConstants.TASK_MANAGER_DATA_PORT_KEY,
    +			ConfigConstants.DEFAULT_TASK_MANAGER_DATA_PORT);
    +		if (dataport == 0) {
    +			dataport = NetUtils.getAvailablePort();
    +		}
    +		checkConfigParameter(dataport > 0, dataport, ConfigConstants.TASK_MANAGER_DATA_PORT_KEY,
    +			"Leave config parameter empty or use 0 to let the system choose a port automatically.");
    +
    +		InetAddress taskManagerAddress = InetAddress.getByName(taskManagerHostname);
    +		InstanceConnectionInfo connectionInfo = new InstanceConnectionInfo(taskManagerAddress, dataport);
    +
    +		// ----> memory / network stack (shuffles/broadcasts), task slots, temp directories
    +
    +		// we need this because many configs have been written with a "-1" entry
    +		int slots = configuration.getInteger(ConfigConstants.TASK_MANAGER_NUM_TASK_SLOTS, 1);
    +		if (slots == -1) {
    +			slots = 1;
    +		}
    +		checkConfigParameter(slots >= 1, slots, ConfigConstants.TASK_MANAGER_NUM_TASK_SLOTS,
    +			"Number of task slots must be at least one.");
    +
    +		int numNetworkBuffers = configuration.getInteger(
    +			ConfigConstants.TASK_MANAGER_NETWORK_NUM_BUFFERS_KEY,
    +			ConfigConstants.DEFAULT_TASK_MANAGER_NETWORK_NUM_BUFFERS);
    +		checkConfigParameter(numNetworkBuffers > 0, numNetworkBuffers,
    +			ConfigConstants.TASK_MANAGER_NETWORK_NUM_BUFFERS_KEY, "");
    +
    +		int pageSize = configuration.getInteger(
    +			ConfigConstants.TASK_MANAGER_MEMORY_SEGMENT_SIZE_KEY,
    +			ConfigConstants.DEFAULT_TASK_MANAGER_MEMORY_SEGMENT_SIZE);
    +		// check page size of for minimum size
    +		checkConfigParameter(pageSize >= MemoryManager.MIN_PAGE_SIZE, pageSize,
    +			ConfigConstants.TASK_MANAGER_MEMORY_SEGMENT_SIZE_KEY,
    +			"Minimum memory segment size is " + MemoryManager.MIN_PAGE_SIZE);
    +		// check page size for power of two
    +		checkConfigParameter(MathUtils.isPowerOf2(pageSize), pageSize,
    +			ConfigConstants.TASK_MANAGER_MEMORY_SEGMENT_SIZE_KEY,
    +			"Memory segment size must be a power of 2.");
    +
    +		// check whether we use heap or off-heap memory
    +		MemoryType memType;
    +		if (configuration.getBoolean(ConfigConstants.TASK_MANAGER_MEMORY_OFF_HEAP_KEY, false)) {
    +			memType = MemoryType.OFF_HEAP;
    +		} else {
    +			memType = MemoryType.HEAP;
    +		}
    +
    +		// initialize the memory segment factory accordingly
    +		if (memType == MemoryType.HEAP) {
    +			if (!MemorySegmentFactory.isInitialized()) {
    +				MemorySegmentFactory.initializeFactory(HeapMemorySegment.FACTORY);
    +			} else if (MemorySegmentFactory.getFactory() != HeapMemorySegment.FACTORY) {
    +				throw new Exception("Memory type is set to heap memory, but memory segment " +
    +					"factory has been initialized for off-heap memory segments");
    +			}
    +		} else {
    +			if (!MemorySegmentFactory.isInitialized()) {
    +				MemorySegmentFactory.initializeFactory(HybridMemorySegment.FACTORY);
    +			} else if (MemorySegmentFactory.getFactory() != HybridMemorySegment.FACTORY) {
    +				throw new Exception("Memory type is set to off-heap memory, but memory segment " +
    +					"factory has been initialized for heap memory segments");
    +			}
    +		}
    +
    +		String[] tmpDirs = configuration.getString(
    +			ConfigConstants.TASK_MANAGER_TMP_DIR_KEY,
    +			ConfigConstants.DEFAULT_TASK_MANAGER_TMP_PATH).split(",|" + File.pathSeparator);
    +
    +		NettyConfig nettyConfig = null;
    +		if (!localTaskManagerCommunication) {
    +			nettyConfig = new NettyConfig(connectionInfo.address(), connectionInfo.dataPort(), pageSize, slots, configuration);
    +		}
    +
    +		// Default spill I/O mode for intermediate results
    +		String syncOrAsync = configuration.getString(
    +			ConfigConstants.TASK_MANAGER_NETWORK_DEFAULT_IO_MODE,
    +			ConfigConstants.DEFAULT_TASK_MANAGER_NETWORK_DEFAULT_IO_MODE);
    +
    +		IOMode ioMode;
    +		if (syncOrAsync == "async") {
    +			ioMode = IOManager.IOMode.ASYNC;
    +		} else {
    +			ioMode = IOManager.IOMode.SYNC;
    +		}
    +
    +		NetworkEnvironmentConfiguration networkConfig = new NetworkEnvironmentConfiguration(
    +			numNetworkBuffers,
    +			pageSize,
    +			memType,
    +			ioMode,
    +			nettyConfig == null ? Option.<NettyConfig>empty() : new Some<>(nettyConfig),
    +			new Tuple2<>(500, 3000));
    +
    +		// ----> timeouts, library caching, profiling
    +
    +		FiniteDuration timeout;
    +		try {
    +			timeout = AkkaUtils.getTimeout(configuration);
    +		} catch (Exception e) {
    +			throw new IllegalArgumentException(
    +				"Invalid format for '" + ConfigConstants.AKKA_ASK_TIMEOUT +
    +					"'.Use formats like '50 s' or '1 min' to specify the timeout.");
    +		}
    +		LOG.info("Messages between TaskManager and JobManager have a max timeout of " + timeout);
    +
    +		long cleanupInterval = configuration.getLong(
    +			ConfigConstants.LIBRARY_CACHE_MANAGER_CLEANUP_INTERVAL,
    +			ConfigConstants.DEFAULT_LIBRARY_CACHE_MANAGER_CLEANUP_INTERVAL) * 1000;
    +
    +		FiniteDuration finiteRegistrationDuration = null;
    +		try {
    +			Duration maxRegistrationDuration = Duration.create(configuration.getString(
    +				ConfigConstants.TASK_MANAGER_MAX_REGISTRATION_DURATION,
    +				ConfigConstants.DEFAULT_TASK_MANAGER_MAX_REGISTRATION_DURATION));
    +			if (maxRegistrationDuration.isFinite()) {
    +				finiteRegistrationDuration = new FiniteDuration(maxRegistrationDuration.toSeconds(), TimeUnit.SECONDS);
    +			}
    +		} catch (NumberFormatException e) {
    +			throw new IllegalArgumentException("Invalid format for parameter " +
    +				ConfigConstants.TASK_MANAGER_MAX_REGISTRATION_DURATION, e);
    +		}
    +
    +		FiniteDuration initialRegistrationPause;
    +		try {
    +			Duration pause = Duration.create(configuration.getString(
    +				ConfigConstants.TASK_MANAGER_INITIAL_REGISTRATION_PAUSE,
    +				ConfigConstants.DEFAULT_TASK_MANAGER_INITIAL_REGISTRATION_PAUSE));
    +			if (pause.isFinite()) {
    +				initialRegistrationPause = new FiniteDuration(pause.toSeconds(), TimeUnit.SECONDS);
    +			} else {
    +				throw new IllegalArgumentException("The initial registration pause must be finite: " + pause);
    +			}
    +		} catch (NumberFormatException e) {
    +			throw new IllegalArgumentException("Invalid format for parameter " +
    +				ConfigConstants.TASK_MANAGER_INITIAL_REGISTRATION_PAUSE, e);
    +		}
    +
    +		FiniteDuration maxRegistrationPause;
    +		try {
    +			Duration pause = Duration.create(configuration.getString(
    +				ConfigConstants.TASK_MANAGER_MAX_REGISTARTION_PAUSE,
    +				ConfigConstants.DEFAULT_TASK_MANAGER_MAX_REGISTRATION_PAUSE));
    +			if (pause.isFinite()) {
    +				maxRegistrationPause = new FiniteDuration(pause.toSeconds(), TimeUnit.SECONDS);
    +			} else {
    +				throw new IllegalArgumentException("The maximum registration pause must be finite: " + pause);
    +			}
    +		} catch (NumberFormatException e) {
    +			throw new IllegalArgumentException("Invalid format for parameter " +
    +				ConfigConstants.TASK_MANAGER_INITIAL_REGISTRATION_PAUSE, e);
    +		}
    +
    +		FiniteDuration refusedRegistrationPause;
    +		try {
    +			Duration pause = Duration.create(configuration.getString(
    +				ConfigConstants.TASK_MANAGER_REFUSED_REGISTRATION_PAUSE,
    +				ConfigConstants.DEFAULT_TASK_MANAGER_REFUSED_REGISTRATION_PAUSE));
    +			if (pause.isFinite()) {
    +				refusedRegistrationPause = new FiniteDuration(pause.toSeconds(), TimeUnit.SECONDS);
    +			} else {
    +				throw new IllegalArgumentException("The refused registration pause must be finite: " + pause);
    +			}
    +		} catch (NumberFormatException e) {
    +			throw new IllegalArgumentException("Invalid format for parameter " +
    +				ConfigConstants.TASK_MANAGER_INITIAL_REGISTRATION_PAUSE, e);
    +		}
    +
    +		TaskManagerConfiguration taskManagerConfig = new TaskManagerConfiguration(
    --- End diff --
    
    Yes, i will do that later, also includes NetworkEnvironmentConfiguration to java.


---
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.
---

[GitHub] flink issue #2400: [FLINK-4363] Implement TaskManager basic startup of all c...

Posted by mxm <gi...@git.apache.org>.
Github user mxm commented on the issue:

    https://github.com/apache/flink/pull/2400
  
    Removing merge commits, rebasing, and merging. 


---
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.
---

[GitHub] flink pull request #2400: [FLINK-4363] Implement TaskManager basic startup o...

Posted by mxm <gi...@git.apache.org>.
Github user mxm commented on a diff in the pull request:

    https://github.com/apache/flink/pull/2400#discussion_r75646940
  
    --- Diff: flink-runtime/src/main/java/org/apache/flink/runtime/rpc/taskexecutor/TaskExecutor.java ---
    @@ -35,27 +79,634 @@
      */
     public class TaskExecutor extends RpcEndpoint<TaskExecutorGateway> {
     
    -	/** The unique resource ID of this TaskExecutor */
    +	private static final Logger LOG = LoggerFactory.getLogger(TaskExecutor.class);
    +
    +	/** Return code for critical errors during the runtime */
    +	private static final int RUNTIME_FAILURE_RETURN_CODE = 2;
    +
    +	/** The name of the TaskManager actor */
    +	private static final String TASK_MANAGER_NAME = "taskmanager";
    +
    +	/** The unique resource ID of this TaskManager */
     	private final ResourceID resourceID;
     
     	/** The access to the leader election and metadata storage services */
     	private final HighAvailabilityServices haServices;
     
    -	// --------- resource manager --------
    +	/** The task manager configuration */
    +	private final TaskManagerConfiguration taskManagerConfig;
    +
    +	/** The connection information of the task manager */
    +	private final InstanceConnectionInfo connectionInfo;
    +
    +	/** The I/O manager component in the task manager */
    +	private final IOManager ioManager;
    +
    +	/** The memory manager component in the task manager */
    +	private final MemoryManager memoryManager;
    +
    +	/** The network component in the task manager */
    +	private final NetworkEnvironment networkEnvironment;
    +
    +	/** The number of slots in the task manager, should be 1 for YARN */
    +	private final int numberOfSlots;
     
    +	// --------- resource manager --------
     	private TaskExecutorToResourceManagerConnection resourceManagerConnection;
     
     	// ------------------------------------------------------------------------
     
     	public TaskExecutor(
    +			TaskManagerConfiguration taskManagerConfig,
    +			ResourceID resourceID,
    +			InstanceConnectionInfo connectionInfo,
    +			MemoryManager memoryManager,
    +			IOManager ioManager,
    +			NetworkEnvironment networkEnvironment,
    +			int numberOfSlots,
     			RpcService rpcService,
    -			HighAvailabilityServices haServices,
    -			ResourceID resourceID) {
    +			HighAvailabilityServices haServices) {
     
     		super(rpcService);
     
    -		this.haServices = checkNotNull(haServices);
    +		this.taskManagerConfig = checkNotNull(taskManagerConfig);
     		this.resourceID = checkNotNull(resourceID);
    +		this.connectionInfo = checkNotNull(connectionInfo);
    +		this.memoryManager = checkNotNull(memoryManager);
    +		this.ioManager = checkNotNull(ioManager);
    +		this.networkEnvironment = checkNotNull(networkEnvironment);
    +		this.numberOfSlots = checkNotNull(numberOfSlots);
    +		this.haServices = checkNotNull(haServices);
    +	}
    +
    +	/**
    +	 * Starts and runs the TaskManager.
    +	 * <p/>
    +	 * This method first tries to select the network interface to use for the TaskManager
    +	 * communication. The network interface is used both for the actor communication
    +	 * (coordination) as well as for the data exchange between task managers. Unless
    +	 * the hostname/interface is explicitly configured in the configuration, this
    +	 * method will try out various interfaces and methods to connect to the JobManager
    +	 * and select the one where the connection attempt is successful.
    +	 * <p/>
    +	 * After selecting the network interface, this method brings up an actor system
    +	 * for the TaskManager and its actors, starts the TaskManager's services
    +	 * (library cache, shuffle network stack, ...), and starts the TaskManager itself.
    +	 *
    +	 * @param configuration    The configuration for the TaskManager.
    +	 * @param taskManagerClass The actor class to instantiate.
    --- End diff --
    
    Missing `ResourceID` param here.


---
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.
---

[GitHub] flink pull request #2400: [FLINK-4363] Implement TaskManager basic startup o...

Posted by mxm <gi...@git.apache.org>.
Github user mxm commented on a diff in the pull request:

    https://github.com/apache/flink/pull/2400#discussion_r76084398
  
    --- Diff: flink-runtime/src/main/java/org/apache/flink/runtime/rpc/taskexecutor/TaskExecutor.java ---
    @@ -36,27 +82,617 @@
      */
     public class TaskExecutor extends RpcEndpoint<TaskExecutorGateway> {
     
    -	/** The unique resource ID of this TaskExecutor */
    +	private static final Logger LOG = LoggerFactory.getLogger(TaskExecutor.class);
    +
    +	/** The unique resource ID of this TaskManager */
     	private final ResourceID resourceID;
     
     	/** The access to the leader election and metadata storage services */
     	private final HighAvailabilityServices haServices;
     
    -	// --------- resource manager --------
    +	/** The task manager configuration */
    +	private final TaskExecutorConfiguration taskExecutorConfig;
    +
    +	/** The connection information of the task manager */
    +	private final InstanceConnectionInfo connectionInfo;
    +
    +	/** The I/O manager component in the task manager */
    +	private final IOManager ioManager;
    +
    +	/** The memory manager component in the task manager */
    +	private final MemoryManager memoryManager;
    +
    +	/** The network component in the task manager */
    +	private final NetworkEnvironment networkEnvironment;
    +
    +	/** The number of slots in the task manager, should be 1 for YARN */
    +	private final int numberOfSlots;
     
    +	// --------- resource manager --------
     	private TaskExecutorToResourceManagerConnection resourceManagerConnection;
     
     	// ------------------------------------------------------------------------
     
     	public TaskExecutor(
    +			TaskExecutorConfiguration taskExecutorConfig,
    +			ResourceID resourceID,
    +			InstanceConnectionInfo connectionInfo,
    +			MemoryManager memoryManager,
    +			IOManager ioManager,
    +			NetworkEnvironment networkEnvironment,
    +			int numberOfSlots,
     			RpcService rpcService,
    -			HighAvailabilityServices haServices,
    -			ResourceID resourceID) {
    +			HighAvailabilityServices haServices) {
     
     		super(rpcService);
     
    -		this.haServices = checkNotNull(haServices);
    +		this.taskExecutorConfig = checkNotNull(taskExecutorConfig);
     		this.resourceID = checkNotNull(resourceID);
    +		this.connectionInfo = checkNotNull(connectionInfo);
    +		this.memoryManager = checkNotNull(memoryManager);
    +		this.ioManager = checkNotNull(ioManager);
    +		this.networkEnvironment = checkNotNull(networkEnvironment);
    +		this.numberOfSlots = checkNotNull(numberOfSlots);
    +		this.haServices = checkNotNull(haServices);
    +	}
    +
    +	/**
    +	 * Starts and runs the TaskManager.
    +	 * <p/>
    +	 * This method first tries to select the network interface to use for the TaskManager
    +	 * communication. The network interface is used both for the actor communication
    +	 * (coordination) as well as for the data exchange between task managers. Unless
    +	 * the hostname/interface is explicitly configured in the configuration, this
    +	 * method will try out various interfaces and methods to connect to the JobManager
    +	 * and select the one where the connection attempt is successful.
    +	 * <p/>
    +	 * After selecting the network interface, this method brings up an actor system
    +	 * for the TaskManager and its actors, starts the TaskManager's services
    +	 * (library cache, shuffle network stack, ...), and starts the TaskManager itself.
    +	 *
    +	 * @param configuration    The configuration for the TaskManager.
    +	 * @param resourceID       The id of the resource which the task manager will run on.
    +	 */
    +	public static void selectNetworkInterfaceAndRunTaskManager(
    +		Configuration configuration,
    +		ResourceID resourceID) throws Exception {
    +
    +		InetSocketAddress taskManagerAddress = selectNetworkInterfaceAndPort(configuration);
    +
    +		runTaskManager(taskManagerAddress.getHostName(), resourceID, taskManagerAddress.getPort(), configuration);
    +	}
    +
    +	private static InetSocketAddress selectNetworkInterfaceAndPort(Configuration configuration)
    +		throws Exception {
    +		String taskManagerHostname = configuration.getString(ConfigConstants.TASK_MANAGER_HOSTNAME_KEY, null);
    +		if (taskManagerHostname != null) {
    +			LOG.info("Using configured hostname/address for TaskManager: " + taskManagerHostname);
    +		} else {
    +			LeaderRetrievalService leaderRetrievalService = LeaderRetrievalUtils.createLeaderRetrievalService(configuration);
    +			FiniteDuration lookupTimeout = AkkaUtils.getLookupTimeout(configuration);
    +
    +			InetAddress taskManagerAddress = LeaderRetrievalUtils.findConnectingAddress(leaderRetrievalService, lookupTimeout);
    +			taskManagerHostname = taskManagerAddress.getHostName();
    +			LOG.info("TaskManager will use hostname/address '{}' ({}) for communication.",
    +				taskManagerHostname, taskManagerAddress.getHostAddress());
    +		}
    +
    +		// if no task manager port has been configured, use 0 (system will pick any free port)
    +		int actorSystemPort = configuration.getInteger(ConfigConstants.TASK_MANAGER_IPC_PORT_KEY, 0);
    +		if (actorSystemPort < 0 || actorSystemPort > 65535) {
    +			throw new IllegalConfigurationException("Invalid value for '" +
    +				ConfigConstants.TASK_MANAGER_IPC_PORT_KEY +
    +				"' (port for the TaskManager actor system) : " + actorSystemPort +
    +				" - Leave config parameter empty or use 0 to let the system choose a port automatically.");
    +		}
    +
    +		return new InetSocketAddress(taskManagerHostname, actorSystemPort);
    +	}
    +
    +	/**
    +	 * Starts and runs the TaskManager. Brings up an actor system for the TaskManager and its
    +	 * actors, starts the TaskManager's services (library cache, shuffle network stack, ...),
    +	 * and starts the TaskManager itself.
    +	 * <p/>
    +	 * This method will also spawn a process reaper for the TaskManager (kill the process if
    +	 * the actor fails) and optionally start the JVM memory logging thread.
    +	 *
    +	 * @param taskManagerHostname The hostname/address of the interface where the actor system
    +	 *                            will communicate.
    +	 * @param resourceID          The id of the resource which the task manager will run on.
    +	 * @param actorSystemPort   The port at which the actor system will communicate.
    +	 * @param configuration       The configuration for the TaskManager.
    +	 */
    +	private static void runTaskManager(
    +		String taskManagerHostname,
    +		ResourceID resourceID,
    +		int actorSystemPort,
    +		final Configuration configuration) throws Exception {
    +
    +		LOG.info("Starting TaskManager");
    +
    +		// Bring up the TaskManager actor system first, bind it to the given address.
    +
    +		LOG.info("Starting TaskManager actor system at " +
    +			NetUtils.hostAndPortToUrlString(taskManagerHostname, actorSystemPort));
    +
    +		final ActorSystem taskManagerSystem;
    +		try {
    +			Tuple2<String, Object> address = new Tuple2<String, Object>(taskManagerHostname, actorSystemPort);
    +			Config akkaConfig = AkkaUtils.getAkkaConfig(configuration, new Some<>(address));
    +			LOG.debug("Using akka configuration\n " + akkaConfig);
    +			taskManagerSystem = AkkaUtils.createActorSystem(akkaConfig);
    +		} catch (Throwable t) {
    +			if (t instanceof org.jboss.netty.channel.ChannelException) {
    +				Throwable cause = t.getCause();
    +				if (cause != null && t.getCause() instanceof java.net.BindException) {
    +					String address = NetUtils.hostAndPortToUrlString(taskManagerHostname, actorSystemPort);
    +					throw new IOException("Unable to bind TaskManager actor system to address " +
    +						address + " - " + cause.getMessage(), t);
    +				}
    +			}
    +			throw new Exception("Could not create TaskManager actor system", t);
    +		}
    +
    +		// start akka rpc service based on actor system
    +		Timeout timeout = new Timeout(AkkaUtils.getTimeout(configuration).toMillis(), TimeUnit.MILLISECONDS);
    +		AkkaRpcService akkaRpcService = new AkkaRpcService(taskManagerSystem, timeout);
    +
    +		// start high availability service to implement getResourceManagerLeaderRetriever method only
    +		HighAvailabilityServices haServices = new HighAvailabilityServices() {
    +			@Override
    +			public LeaderRetrievalService getResourceManagerLeaderRetriever() throws Exception {
    +				return LeaderRetrievalUtils.createLeaderRetrievalService(configuration);
    +			}
    +
    +			@Override
    +			public LeaderElectionService getJobMasterLeaderElectionService(JobID jobID) throws Exception {
    +				return null;
    +			}
    +		};
    +
    +		// start all the TaskManager services (network stack,  library cache, ...)
    +		// and the TaskManager actor
    +		try {
    +			LOG.info("Starting TaskManager actor");
    +			TaskExecutor taskExecutor = startTaskManagerComponentsAndActor(
    +				configuration,
    +				resourceID,
    +				akkaRpcService,
    +				taskManagerHostname,
    +				haServices,
    +				false);
    +
    +			taskExecutor.start();
    +
    +			// if desired, start the logging daemon that periodically logs the memory usage information
    +			if (LOG.isInfoEnabled() && configuration.getBoolean(
    +				ConfigConstants.TASK_MANAGER_DEBUG_MEMORY_USAGE_START_LOG_THREAD,
    +				ConfigConstants.DEFAULT_TASK_MANAGER_DEBUG_MEMORY_USAGE_START_LOG_THREAD)) {
    +				LOG.info("Starting periodic memory usage logger");
    +
    +				long interval = configuration.getLong(
    +					ConfigConstants.TASK_MANAGER_DEBUG_MEMORY_USAGE_LOG_INTERVAL_MS,
    +					ConfigConstants.DEFAULT_TASK_MANAGER_DEBUG_MEMORY_USAGE_LOG_INTERVAL_MS);
    +
    +				MemoryLogger logger = new MemoryLogger(LOG, interval, taskManagerSystem);
    +				logger.start();
    +			}
    +
    +			// block until everything is done
    +			taskManagerSystem.awaitTermination();
    +		} catch (Throwable t) {
    +			LOG.error("Error while starting up taskManager", t);
    +			try {
    +				taskManagerSystem.shutdown();
    +			} catch (Throwable tt) {
    +				LOG.warn("Could not cleanly shut down actor system", tt);
    +			}
    +			throw t;
    +		}
    +	}
    +
    +	// --------------------------------------------------------------------------
    +	//  Starting and running the TaskManager
    +	// --------------------------------------------------------------------------
    +
    +	/**
    +	 * @param configuration                 The configuration for the TaskManager.
    +	 * @param resourceID                    The id of the resource which the task manager will run on.
    +	 * @param rpcService                  The rpc service which is used to start and connect to the TaskManager RpcEndpoint .
    +	 * @param taskManagerHostname       The hostname/address that describes the TaskManager's data location.
    +	 * @param haServices        Optionally, a high availability service can be provided. If none is given,
    +	 *                                      then a HighAvailabilityServices is constructed from the configuration.
    +	 * @param localTaskManagerCommunication If true, the TaskManager will not initiate the TCP network stack.
    +	 * @return An ActorRef to the TaskManager actor.
    +	 * @throws org.apache.flink.configuration.IllegalConfigurationException     Thrown, if the given config contains illegal values.
    +	 * @throws java.io.IOException      Thrown, if any of the I/O components (such as buffer pools,
    +	 *                                       I/O manager, ...) cannot be properly started.
    +	 * @throws java.lang.Exception      Thrown is some other error occurs while parsing the configuration
    +	 *                                      or starting the TaskManager components.
    +	 */
    +	public static TaskExecutor startTaskManagerComponentsAndActor(
    +		Configuration configuration,
    +		ResourceID resourceID,
    +		RpcService rpcService,
    +		String taskManagerHostname,
    +		HighAvailabilityServices haServices,
    +		boolean localTaskManagerCommunication) throws Exception {
    +
    +		Tuple4<TaskExecutorConfiguration, NetworkEnvironmentConfiguration, InstanceConnectionInfo, MemoryType> tuple4
    +			= parseTaskManagerConfiguration(configuration, taskManagerHostname, localTaskManagerCommunication);
    +
    +		TaskExecutorConfiguration taskExecutorConfig = tuple4.f0;
    +		NetworkEnvironmentConfiguration netConfig = tuple4.f1;
    +		InstanceConnectionInfo connectionInfo = tuple4.f2;
    +		MemoryType memType = tuple4.f3;
    +
    +		// pre-start checks
    +		checkTempDirs(taskExecutorConfig.getTmpDirPaths());
    +
    +		ExecutionContext executionContext = ExecutionContexts$.MODULE$.fromExecutor(new ForkJoinPool());
    +
    +		// we start the network first, to make sure it can allocate its buffers first
    +		NetworkEnvironment network = new NetworkEnvironment(executionContext, taskExecutorConfig.getTimeout(), netConfig);
    +
    +		// computing the amount of memory to use depends on how much memory is available
    +		// it strictly needs to happen AFTER the network stack has been initialized
    +
    +		// check if a value has been configured
    +		long configuredMemory = configuration.getLong(ConfigConstants.TASK_MANAGER_MEMORY_SIZE_KEY, -1L);
    +		checkConfigParameter(configuredMemory == -1 || configuredMemory > 0, configuredMemory,
    +			ConfigConstants.TASK_MANAGER_MEMORY_SIZE_KEY,
    +			"MemoryManager needs at least one MB of memory. " +
    +				"If you leave this config parameter empty, the system automatically " +
    +				"pick a fraction of the available memory.");
    +
    +		long memorySize;
    +		boolean preAllocateMemory = configuration.getBoolean(
    +			ConfigConstants.TASK_MANAGER_MEMORY_PRE_ALLOCATE_KEY,
    +			ConfigConstants.DEFAULT_TASK_MANAGER_MEMORY_PRE_ALLOCATE);
    +		if (configuredMemory > 0) {
    +			if (preAllocateMemory) {
    +				LOG.info("Using {} MB for managed memory." , configuredMemory);
    +			} else {
    +				LOG.info("Limiting managed memory to {} MB, memory will be allocated lazily." , configuredMemory);
    +			}
    +			memorySize = configuredMemory << 20; // megabytes to bytes
    +		} else {
    +			float fraction = configuration.getFloat(
    +				ConfigConstants.TASK_MANAGER_MEMORY_FRACTION_KEY,
    +				ConfigConstants.DEFAULT_MEMORY_MANAGER_MEMORY_FRACTION);
    +			checkConfigParameter(fraction > 0.0f && fraction < 1.0f, fraction,
    +				ConfigConstants.TASK_MANAGER_MEMORY_FRACTION_KEY,
    +				"MemoryManager fraction of the free memory must be between 0.0 and 1.0");
    +
    +			if (memType == MemoryType.HEAP) {
    +				long relativeMemSize = (long) (EnvironmentInformation.getSizeOfFreeHeapMemoryWithDefrag() * fraction);
    +				if (preAllocateMemory) {
    +					LOG.info("Using {} of the currently free heap space for managed heap memory ({} MB)." ,
    +						fraction , relativeMemSize >> 20);
    +				} else {
    +					LOG.info("Limiting managed memory to {} of the currently free heap space ({} MB), " +
    +						"memory will be allocated lazily." , fraction , relativeMemSize >> 20);
    +				}
    +				memorySize = relativeMemSize;
    +			} else if (memType == MemoryType.OFF_HEAP) {
    +				// The maximum heap memory has been adjusted according to the fraction
    +				long maxMemory = EnvironmentInformation.getMaxJvmHeapMemory();
    +				long directMemorySize = (long) (maxMemory / (1.0 - fraction) * fraction);
    +				if (preAllocateMemory) {
    +					LOG.info("Using {} of the maximum memory size for managed off-heap memory ({} MB)." ,
    +						fraction, directMemorySize >> 20);
    +				} else {
    +					LOG.info("Limiting managed memory to {} of the maximum memory size ({} MB)," +
    +						" memory will be allocated lazily.", fraction, directMemorySize >> 20);
    +				}
    +				memorySize = directMemorySize;
    +			} else {
    +				throw new RuntimeException("No supported memory type detected.");
    +			}
    +		}
    +
    +		// now start the memory manager
    +		final MemoryManager memoryManager;
    +		try {
    +			memoryManager = new MemoryManager(
    +				memorySize,
    +				taskExecutorConfig.getNumberOfSlots(),
    +				netConfig.networkBufferSize(),
    +				memType,
    +				preAllocateMemory);
    +		} catch (OutOfMemoryError e) {
    +			if (memType == MemoryType.HEAP) {
    +				throw new Exception("OutOfMemory error (" + e.getMessage() +
    +					") while allocating the TaskManager heap memory (" + memorySize + " bytes).", e);
    +			} else if (memType == MemoryType.OFF_HEAP) {
    +				throw new Exception("OutOfMemory error (" + e.getMessage() +
    +					") while allocating the TaskManager off-heap memory (" + memorySize +
    +					" bytes).Try increasing the maximum direct memory (-XX:MaxDirectMemorySize)", e);
    +			} else {
    +				throw e;
    +			}
    +		}
    +
    +		// start the I/O manager, it will create some temp directories.
    +		IOManager ioManager = new IOManagerAsync(taskExecutorConfig.getTmpDirPaths());
    +
    +		TaskExecutor taskExecutor = new TaskExecutor(
    +			taskExecutorConfig,
    +			resourceID,
    +			connectionInfo,
    +			memoryManager,
    +			ioManager,
    +			network,
    +			taskExecutorConfig.getNumberOfSlots(),
    +			rpcService,
    +			haServices);
    +
    +		return taskExecutor;
    +	}
    +
    +	// --------------------------------------------------------------------------
    +	//  Parsing and checking the TaskManager Configuration
    +	// --------------------------------------------------------------------------
    +
    +	/**
    +	 * Utility method to extract TaskManager config parameters from the configuration and to
    +	 * sanity check them.
    +	 *
    +	 * @param configuration                 The configuration.
    +	 * @param taskManagerHostname           The host name under which the TaskManager communicates.
    +	 * @param localTaskManagerCommunication True, to skip initializing the network stack.
    +	 *                                      Use only in cases where only one task manager runs.
    +	 * @return A tuple (TaskManagerConfiguration, network configuration,
    +	 * InstanceConnectionInfo, JobManager actor Akka URL).
    +	 */
    +	private static Tuple4<TaskExecutorConfiguration, NetworkEnvironmentConfiguration, InstanceConnectionInfo, MemoryType>
    +		parseTaskManagerConfiguration(Configuration configuration, String taskManagerHostname, boolean localTaskManagerCommunication)
    +		throws Exception {
    +
    +		// ------- read values from the config and check them ---------
    +		//                      (a lot of them)
    +
    +		// ----> hosts / ports for communication and data exchange
    +
    +		int dataport = configuration.getInteger(ConfigConstants.TASK_MANAGER_DATA_PORT_KEY,
    +			ConfigConstants.DEFAULT_TASK_MANAGER_DATA_PORT);
    +		if (dataport == 0) {
    +			dataport = NetUtils.getAvailablePort();
    +		}
    +		checkConfigParameter(dataport > 0, dataport, ConfigConstants.TASK_MANAGER_DATA_PORT_KEY,
    +			"Leave config parameter empty or use 0 to let the system choose a port automatically.");
    +
    +		InetAddress taskManagerAddress = InetAddress.getByName(taskManagerHostname);
    +		InstanceConnectionInfo connectionInfo = new InstanceConnectionInfo(taskManagerAddress, dataport);
    +
    +		// ----> memory / network stack (shuffles/broadcasts), task slots, temp directories
    +
    +		// we need this because many configs have been written with a "-1" entry
    +		int slots = configuration.getInteger(ConfigConstants.TASK_MANAGER_NUM_TASK_SLOTS, 1);
    +		if (slots == -1) {
    +			slots = 1;
    +		}
    +		checkConfigParameter(slots >= 1, slots, ConfigConstants.TASK_MANAGER_NUM_TASK_SLOTS,
    +			"Number of task slots must be at least one.");
    +
    +		int numNetworkBuffers = configuration.getInteger(
    +			ConfigConstants.TASK_MANAGER_NETWORK_NUM_BUFFERS_KEY,
    +			ConfigConstants.DEFAULT_TASK_MANAGER_NETWORK_NUM_BUFFERS);
    +		checkConfigParameter(numNetworkBuffers > 0, numNetworkBuffers,
    +			ConfigConstants.TASK_MANAGER_NETWORK_NUM_BUFFERS_KEY, "");
    +
    +		int pageSize = configuration.getInteger(
    +			ConfigConstants.TASK_MANAGER_MEMORY_SEGMENT_SIZE_KEY,
    +			ConfigConstants.DEFAULT_TASK_MANAGER_MEMORY_SEGMENT_SIZE);
    +		// check page size of for minimum size
    +		checkConfigParameter(pageSize >= MemoryManager.MIN_PAGE_SIZE, pageSize,
    +			ConfigConstants.TASK_MANAGER_MEMORY_SEGMENT_SIZE_KEY,
    +			"Minimum memory segment size is " + MemoryManager.MIN_PAGE_SIZE);
    +		// check page size for power of two
    +		checkConfigParameter(MathUtils.isPowerOf2(pageSize), pageSize,
    +			ConfigConstants.TASK_MANAGER_MEMORY_SEGMENT_SIZE_KEY,
    +			"Memory segment size must be a power of 2.");
    +
    +		// check whether we use heap or off-heap memory
    +		final MemoryType memType;
    +		if (configuration.getBoolean(ConfigConstants.TASK_MANAGER_MEMORY_OFF_HEAP_KEY, false)) {
    +			memType = MemoryType.OFF_HEAP;
    +		} else {
    +			memType = MemoryType.HEAP;
    +		}
    +
    +		// initialize the memory segment factory accordingly
    +		if (memType == MemoryType.HEAP) {
    +			if (!MemorySegmentFactory.isInitialized()) {
    +				MemorySegmentFactory.initializeFactory(HeapMemorySegment.FACTORY);
    +			} else if (MemorySegmentFactory.getFactory() != HeapMemorySegment.FACTORY) {
    +				throw new Exception("Memory type is set to heap memory, but memory segment " +
    +					"factory has been initialized for off-heap memory segments");
    +			}
    +		} else {
    +			if (!MemorySegmentFactory.isInitialized()) {
    +				MemorySegmentFactory.initializeFactory(HybridMemorySegment.FACTORY);
    +			} else if (MemorySegmentFactory.getFactory() != HybridMemorySegment.FACTORY) {
    +				throw new Exception("Memory type is set to off-heap memory, but memory segment " +
    +					"factory has been initialized for heap memory segments");
    +			}
    +		}
    +
    +		String[] tmpDirs = configuration.getString(
    +			ConfigConstants.TASK_MANAGER_TMP_DIR_KEY,
    +			ConfigConstants.DEFAULT_TASK_MANAGER_TMP_PATH).split(",|" + File.pathSeparator);
    +
    +		final NettyConfig nettyConfig;
    +		if (!localTaskManagerCommunication) {
    +			nettyConfig = new NettyConfig(connectionInfo.address(), connectionInfo.dataPort(), pageSize, slots, configuration);
    +		} else {
    +			nettyConfig = null;
    +		}
    +
    +		// Default spill I/O mode for intermediate results
    +		String syncOrAsync = configuration.getString(
    +			ConfigConstants.TASK_MANAGER_NETWORK_DEFAULT_IO_MODE,
    +			ConfigConstants.DEFAULT_TASK_MANAGER_NETWORK_DEFAULT_IO_MODE);
    +
    +		final IOMode ioMode;
    +		if (syncOrAsync == "async") {
    --- End diff --
    
    This doesn't work in Java. You need to use `equals(..)`. Or in this case you can use `syncOrAsync == IOMode.ASYNC`.


---
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.
---

[GitHub] flink pull request #2400: [FLINK-4363] Implement TaskManager basic startup o...

Posted by wangzhijiang999 <gi...@git.apache.org>.
Github user wangzhijiang999 commented on a diff in the pull request:

    https://github.com/apache/flink/pull/2400#discussion_r75651382
  
    --- Diff: flink-runtime/src/main/java/org/apache/flink/runtime/rpc/taskexecutor/TaskExecutor.java ---
    @@ -35,27 +79,634 @@
      */
     public class TaskExecutor extends RpcEndpoint<TaskExecutorGateway> {
     
    -	/** The unique resource ID of this TaskExecutor */
    +	private static final Logger LOG = LoggerFactory.getLogger(TaskExecutor.class);
    +
    +	/** Return code for critical errors during the runtime */
    +	private static final int RUNTIME_FAILURE_RETURN_CODE = 2;
    +
    +	/** The name of the TaskManager actor */
    +	private static final String TASK_MANAGER_NAME = "taskmanager";
    +
    +	/** The unique resource ID of this TaskManager */
     	private final ResourceID resourceID;
     
     	/** The access to the leader election and metadata storage services */
     	private final HighAvailabilityServices haServices;
     
    -	// --------- resource manager --------
    +	/** The task manager configuration */
    +	private final TaskManagerConfiguration taskManagerConfig;
    +
    +	/** The connection information of the task manager */
    +	private final InstanceConnectionInfo connectionInfo;
    +
    +	/** The I/O manager component in the task manager */
    +	private final IOManager ioManager;
    +
    +	/** The memory manager component in the task manager */
    +	private final MemoryManager memoryManager;
    +
    +	/** The network component in the task manager */
    +	private final NetworkEnvironment networkEnvironment;
    +
    +	/** The number of slots in the task manager, should be 1 for YARN */
    +	private final int numberOfSlots;
     
    +	// --------- resource manager --------
     	private TaskExecutorToResourceManagerConnection resourceManagerConnection;
     
     	// ------------------------------------------------------------------------
     
     	public TaskExecutor(
    +			TaskManagerConfiguration taskManagerConfig,
    +			ResourceID resourceID,
    +			InstanceConnectionInfo connectionInfo,
    +			MemoryManager memoryManager,
    +			IOManager ioManager,
    +			NetworkEnvironment networkEnvironment,
    +			int numberOfSlots,
     			RpcService rpcService,
    -			HighAvailabilityServices haServices,
    -			ResourceID resourceID) {
    +			HighAvailabilityServices haServices) {
     
     		super(rpcService);
     
    -		this.haServices = checkNotNull(haServices);
    +		this.taskManagerConfig = checkNotNull(taskManagerConfig);
     		this.resourceID = checkNotNull(resourceID);
    +		this.connectionInfo = checkNotNull(connectionInfo);
    +		this.memoryManager = checkNotNull(memoryManager);
    +		this.ioManager = checkNotNull(ioManager);
    +		this.networkEnvironment = checkNotNull(networkEnvironment);
    +		this.numberOfSlots = checkNotNull(numberOfSlots);
    +		this.haServices = checkNotNull(haServices);
    +	}
    +
    +	/**
    +	 * Starts and runs the TaskManager.
    +	 * <p/>
    +	 * This method first tries to select the network interface to use for the TaskManager
    +	 * communication. The network interface is used both for the actor communication
    +	 * (coordination) as well as for the data exchange between task managers. Unless
    +	 * the hostname/interface is explicitly configured in the configuration, this
    +	 * method will try out various interfaces and methods to connect to the JobManager
    +	 * and select the one where the connection attempt is successful.
    +	 * <p/>
    +	 * After selecting the network interface, this method brings up an actor system
    +	 * for the TaskManager and its actors, starts the TaskManager's services
    +	 * (library cache, shuffle network stack, ...), and starts the TaskManager itself.
    +	 *
    +	 * @param configuration    The configuration for the TaskManager.
    +	 * @param taskManagerClass The actor class to instantiate.
    +	 *                         Allows to use TaskManager subclasses for example for YARN.
    +	 */
    +	public static void selectNetworkInterfaceAndRunTaskManager(
    +		Configuration configuration,
    +		ResourceID resourceID,
    +		Class<? extends TaskManager> taskManagerClass) throws Exception {
    +
    +		Tuple2<String, Integer> tuple2 = selectNetworkInterfaceAndPort(configuration);
    +
    +		runTaskManager(tuple2._1(), resourceID, tuple2._2(), configuration, taskManagerClass);
    +	}
    +
    +	private static Tuple2<String, Integer> selectNetworkInterfaceAndPort(Configuration configuration)
    +		throws Exception {
    +		String taskManagerHostname = configuration.getString(ConfigConstants.TASK_MANAGER_HOSTNAME_KEY, null);
    +		if (taskManagerHostname != null) {
    +			LOG.info("Using configured hostname/address for TaskManager: " + taskManagerHostname);
    +		} else {
    +			LeaderRetrievalService leaderRetrievalService = LeaderRetrievalUtils.createLeaderRetrievalService(configuration);
    +			FiniteDuration lookupTimeout = AkkaUtils.getLookupTimeout(configuration);
    +
    +			InetAddress taskManagerAddress = LeaderRetrievalUtils.findConnectingAddress(leaderRetrievalService, lookupTimeout);
    +			taskManagerHostname = taskManagerAddress.getHostName();
    +			LOG.info("TaskManager will use hostname/address '{}' ({}) for communication.",
    +				taskManagerHostname, taskManagerAddress.getHostAddress());
    +		}
    +
    +		// if no task manager port has been configured, use 0 (system will pick any free port)
    +		int actorSystemPort = configuration.getInteger(ConfigConstants.TASK_MANAGER_IPC_PORT_KEY, 0);
    +		if (actorSystemPort < 0 || actorSystemPort > 65535) {
    +			throw new IllegalConfigurationException("Invalid value for '" +
    +				ConfigConstants.TASK_MANAGER_IPC_PORT_KEY +
    +				"' (port for the TaskManager actor system) : " + actorSystemPort +
    +				" - Leave config parameter empty or use 0 to let the system choose a port automatically.");
    +		}
    +
    +		return new Tuple2<>(taskManagerHostname, actorSystemPort);
    +	}
    +
    +	/**
    +	 * Starts and runs the TaskManager. Brings up an actor system for the TaskManager and its
    +	 * actors, starts the TaskManager's services (library cache, shuffle network stack, ...),
    +	 * and starts the TaskManager itself.
    +	 * <p/>
    +	 * This method will also spawn a process reaper for the TaskManager (kill the process if
    +	 * the actor fails) and optionally start the JVM memory logging thread.
    +	 *
    +	 * @param taskManagerHostname The hostname/address of the interface where the actor system
    +	 *                            will communicate.
    +	 * @param resourceID          The id of the resource which the task manager will run on.
    +	 * @param actorSystemPort     The port at which the actor system will communicate.
    +	 * @param configuration       The configuration for the TaskManager.
    +	 * @param taskManagerClass    The actor class to instantiate. Allows the use of TaskManager
    +	 *                            subclasses for example for YARN.
    +	 */
    +	private static void runTaskManager(
    +		String taskManagerHostname,
    +		ResourceID resourceID,
    +		int actorSystemPort,
    +		Configuration configuration,
    +		Class<? extends TaskManager> taskManagerClass) throws Exception {
    +
    +		LOG.info("Starting TaskManager");
    +
    +		// Bring up the TaskManager actor system first, bind it to the given address.
    +
    +		LOG.info("Starting TaskManager actor system at " +
    +			NetUtils.hostAndPortToUrlString(taskManagerHostname, actorSystemPort));
    +
    +		ActorSystem taskManagerSystem;
    +		try {
    +			Tuple2<String, Object> address = new Tuple2<String, Object>(taskManagerHostname, actorSystemPort);
    +			Config akkaConfig = AkkaUtils.getAkkaConfig(configuration, new Some<>(address));
    +			LOG.debug("Using akka configuration\n " + akkaConfig);
    +			taskManagerSystem = AkkaUtils.createActorSystem(akkaConfig);
    +		} catch (Throwable t) {
    +			if (t instanceof org.jboss.netty.channel.ChannelException) {
    +				Throwable cause = t.getCause();
    +				if (cause != null && t.getCause() instanceof java.net.BindException) {
    +					String address = NetUtils.hostAndPortToUrlString(taskManagerHostname, actorSystemPort);
    +					throw new IOException("Unable to bind TaskManager actor system to address " +
    +						address + " - " + cause.getMessage(), t);
    +				}
    +			}
    +			throw new Exception("Could not create TaskManager actor system", t);
    +		}
    +
    +		// start all the TaskManager services (network stack,  library cache, ...)
    +		// and the TaskManager actor
    +		try {
    +			LOG.info("Starting TaskManager actor");
    +			ActorRef taskManagerActor = startTaskManagerComponentsAndActor(
    +				configuration,
    +				resourceID,
    +				taskManagerSystem,
    +				taskManagerHostname,
    +				TASK_MANAGER_NAME,
    +				null,
    +				false,
    +				taskManagerClass);
    +
    +			// start a process reaper that watches the JobManager. If the TaskManager actor dies,
    +			// the process reaper will kill the JVM process (to ensure easy failure detection)
    +			LOG.debug("Starting TaskManager process reaper");
    +
    +			taskManagerSystem.actorOf(
    +				Props.create(ProcessReaper.class, taskManagerActor, LOG, RUNTIME_FAILURE_RETURN_CODE),
    +				"TaskManager_Process_Reaper");
    +
    +			// if desired, start the logging daemon that periodically logs the memory usage information
    +			if (LOG.isInfoEnabled() && configuration.getBoolean(
    +				ConfigConstants.TASK_MANAGER_DEBUG_MEMORY_USAGE_START_LOG_THREAD,
    +				ConfigConstants.DEFAULT_TASK_MANAGER_DEBUG_MEMORY_USAGE_START_LOG_THREAD)) {
    +				LOG.info("Starting periodic memory usage logger");
    +
    +				long interval = configuration.getLong(
    +					ConfigConstants.TASK_MANAGER_DEBUG_MEMORY_USAGE_LOG_INTERVAL_MS,
    +					ConfigConstants.DEFAULT_TASK_MANAGER_DEBUG_MEMORY_USAGE_LOG_INTERVAL_MS);
    +
    +				MemoryLogger logger = new MemoryLogger(LOG, interval, taskManagerSystem);
    +				logger.start();
    +			}
    +
    +			// block until everything is done
    +			taskManagerSystem.awaitTermination();
    +		} catch (Throwable t) {
    +			LOG.error("Error while starting up taskManager", t);
    +			try {
    +				taskManagerSystem.shutdown();
    +			} catch (Throwable tt) {
    +				LOG.warn("Could not cleanly shut down actor system", tt);
    +			}
    +			throw t;
    +		}
    +	}
    +
    +	// --------------------------------------------------------------------------
    +	//  Starting and running the TaskManager
    +	// --------------------------------------------------------------------------
    +
    +	/**
    +	 * @param configuration                 The configuration for the TaskManager.
    +	 * @param resourceID                    The id of the resource which the task manager will run on.
    +	 * @param actorSystem                  The actor system that should run the TaskManager actor.
    +	 * @param taskManagerHostname       The hostname/address that describes the TaskManager's data location.
    +	 * @param taskManagerActorName      Optionally the name of the TaskManager actor. If none is given,
    +	 *                                      the actor will use a random name.
    +	 * @param leaderRetrievalService        Optionally, a leader retrieval service can be provided. If none is given,
    +	 *                                      then a LeaderRetrievalService is constructed from the configuration.
    +	 * @param localTaskManagerCommunication If true, the TaskManager will not initiate the TCP network stack.
    +	 * @param taskManagerClass      The class of the TaskManager actor. May be used to give
    +	 *                                      subclasses that understand additional actor messages.
    +	 * @return An ActorRef to the TaskManager actor.
    +	 * @throws org.apache.flink.configuration.IllegalConfigurationException     Thrown, if the given config contains illegal values.
    +	 * @throws java.io.IOException      Thrown, if any of the I/O components (such as buffer pools,
    +	 *                                       I/O manager, ...) cannot be properly started.
    +	 * @throws java.lang.Exception      Thrown is some other error occurs while parsing the configuration
    +	 *                                      or starting the TaskManager components.
    +	 */
    +	public static ActorRef startTaskManagerComponentsAndActor(
    +		Configuration configuration,
    +		ResourceID resourceID,
    +		ActorSystem actorSystem,
    +		String taskManagerHostname,
    +		String taskManagerActorName,
    +		LeaderRetrievalService leaderRetrievalService,
    +		boolean localTaskManagerCommunication,
    +		Class<? extends TaskManager> taskManagerClass) throws Exception {
    +
    +		Tuple4<TaskManagerConfiguration, NetworkEnvironmentConfiguration, InstanceConnectionInfo, MemoryType> tuple4
    +			= parseTaskManagerConfiguration(configuration, taskManagerHostname, localTaskManagerCommunication);
    +
    +		TaskManagerConfiguration taskManagerConfig = tuple4._1();
    +		NetworkEnvironmentConfiguration netConfig = tuple4._2();
    +		InstanceConnectionInfo connectionInfo = tuple4._3();
    +		MemoryType memType = tuple4._4();
    +
    +		// pre-start checks
    +		checkTempDirs(taskManagerConfig.tmpDirPaths());
    +
    +		ExecutionContext executionContext = ExecutionContexts$.MODULE$.fromExecutor(new ForkJoinPool());
    +
    +		// we start the network first, to make sure it can allocate its buffers first
    +		NetworkEnvironment network = new NetworkEnvironment(executionContext, taskManagerConfig.timeout(), netConfig);
    +
    +		// computing the amount of memory to use depends on how much memory is available
    +		// it strictly needs to happen AFTER the network stack has been initialized
    +
    +		// check if a value has been configured
    +		long configuredMemory = configuration.getLong(ConfigConstants.TASK_MANAGER_MEMORY_SIZE_KEY, -1L);
    +		checkConfigParameter(configuredMemory == -1 || configuredMemory > 0, configuredMemory,
    +			ConfigConstants.TASK_MANAGER_MEMORY_SIZE_KEY,
    +			"MemoryManager needs at least one MB of memory. " +
    +				"If you leave this config parameter empty, the system automatically " +
    +				"pick a fraction of the available memory.");
    +
    +		long memorySize;
    +		boolean preAllocateMemory = configuration.getBoolean(
    +			ConfigConstants.TASK_MANAGER_MEMORY_PRE_ALLOCATE_KEY,
    +			ConfigConstants.DEFAULT_TASK_MANAGER_MEMORY_PRE_ALLOCATE);
    +		if (configuredMemory > 0) {
    +			if (preAllocateMemory) {
    +				LOG.info("Using {} MB for managed memory." , configuredMemory);
    +			} else {
    +				LOG.info("Limiting managed memory to {} MB, memory will be allocated lazily." , configuredMemory);
    +			}
    +			memorySize = configuredMemory << 20; // megabytes to bytes
    +		} else {
    +			float fraction = configuration.getFloat(
    +				ConfigConstants.TASK_MANAGER_MEMORY_FRACTION_KEY,
    +				ConfigConstants.DEFAULT_MEMORY_MANAGER_MEMORY_FRACTION);
    +			checkConfigParameter(fraction > 0.0f && fraction < 1.0f, fraction,
    +				ConfigConstants.TASK_MANAGER_MEMORY_FRACTION_KEY,
    +				"MemoryManager fraction of the free memory must be between 0.0 and 1.0");
    +
    +			if (memType == MemoryType.HEAP) {
    +				long relativeMemSize = (long) (EnvironmentInformation.getSizeOfFreeHeapMemoryWithDefrag() * fraction);
    +				if (preAllocateMemory) {
    +					LOG.info("Using {} of the currently free heap space for managed heap memory ({} MB)." ,
    +						fraction , relativeMemSize >> 20);
    +				} else {
    +					LOG.info("Limiting managed memory to {} of the currently free heap space ({} MB), " +
    +						"memory will be allocated lazily." , fraction , relativeMemSize >> 20);
    +				}
    +				memorySize = relativeMemSize;
    +			} else if (memType == MemoryType.OFF_HEAP) {
    +				// The maximum heap memory has been adjusted according to the fraction
    +				long maxMemory = EnvironmentInformation.getMaxJvmHeapMemory();
    +				long directMemorySize = (long) (maxMemory / (1.0 - fraction) * fraction);
    +				if (preAllocateMemory) {
    +					LOG.info("Using {} of the maximum memory size for managed off-heap memory ({} MB)." ,
    +						fraction, directMemorySize >> 20);
    +				} else {
    +					LOG.info("Limiting managed memory to {} of the maximum memory size ({} MB)," +
    +						" memory will be allocated lazily.", fraction, directMemorySize >> 20);
    +				}
    +				memorySize = directMemorySize;
    +			} else {
    +				throw new RuntimeException("No supported memory type detected.");
    +			}
    +		}
    +
    +		// now start the memory manager
    +		MemoryManager memoryManager;
    +		try {
    +			memoryManager = new MemoryManager(
    +				memorySize,
    +				taskManagerConfig.numberOfSlots(),
    +				netConfig.networkBufferSize(),
    +				memType,
    +				preAllocateMemory);
    +		} catch (OutOfMemoryError e) {
    +			if (memType == MemoryType.HEAP) {
    +				throw new Exception("OutOfMemory error (" + e.getMessage() +
    +					") while allocating the TaskManager heap memory (" + memorySize + " bytes).", e);
    +			} else if (memType == MemoryType.OFF_HEAP) {
    +				throw new Exception("OutOfMemory error (" + e.getMessage() +
    +					") while allocating the TaskManager off-heap memory (" + memorySize +
    +					" bytes).Try increasing the maximum direct memory (-XX:MaxDirectMemorySize)", e);
    +			} else {
    +				throw e;
    +			}
    +		}
    +
    +		// start the I/O manager, it will create some temp directories.
    +		IOManager ioManager = new IOManagerAsync(taskManagerConfig.tmpDirPaths());
    +
    +		if (leaderRetrievalService == null){
    +			leaderRetrievalService = LeaderRetrievalUtils.createLeaderRetrievalService(configuration);
    +		}
    +
    +		// create the actor properties (which define the actor constructor parameters)
    +		Props tmProps = Props.create(
    +			taskManagerClass,
    +			taskManagerConfig,
    +			resourceID,
    +			connectionInfo,
    +			memoryManager,
    +			ioManager,
    +			network,
    +			taskManagerConfig.numberOfSlots(),
    +			leaderRetrievalService);
    +
    +		ActorRef taskManagerActorRef;
    +		if (taskManagerActorName != null && !taskManagerActorName.equals("")) {
    +			taskManagerActorRef = actorSystem.actorOf(tmProps, taskManagerActorName);
    +		} else {
    +			taskManagerActorRef = actorSystem.actorOf(tmProps);
    +		}
    +
    +		return taskManagerActorRef;
    +	}
    +
    +	// --------------------------------------------------------------------------
    +	//  Parsing and checking the TaskManager Configuration
    +	// --------------------------------------------------------------------------
    +
    +	/**
    +	 * Utility method to extract TaskManager config parameters from the configuration and to
    +	 * sanity check them.
    +	 *
    +	 * @param configuration                 The configuration.
    +	 * @param taskManagerHostname           The host name under which the TaskManager communicates.
    +	 * @param localTaskManagerCommunication True, to skip initializing the network stack.
    +	 *                                      Use only in cases where only one task manager runs.
    +	 * @return A tuple (TaskManagerConfiguration, network configuration,
    +	 * InstanceConnectionInfo, JobManager actor Akka URL).
    +	 */
    +	private static Tuple4<TaskManagerConfiguration, NetworkEnvironmentConfiguration, InstanceConnectionInfo, MemoryType>
    --- End diff --
    
    Yes, it is better to do 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.
---

[GitHub] flink pull request #2400: [FLINK-4363] Implement TaskManager basic startup o...

Posted by mxm <gi...@git.apache.org>.
Github user mxm commented on a diff in the pull request:

    https://github.com/apache/flink/pull/2400#discussion_r76084947
  
    --- Diff: flink-runtime/src/main/java/org/apache/flink/runtime/rpc/taskexecutor/TaskExecutor.java ---
    @@ -36,27 +82,617 @@
      */
     public class TaskExecutor extends RpcEndpoint<TaskExecutorGateway> {
     
    -	/** The unique resource ID of this TaskExecutor */
    +	private static final Logger LOG = LoggerFactory.getLogger(TaskExecutor.class);
    +
    +	/** The unique resource ID of this TaskManager */
     	private final ResourceID resourceID;
     
     	/** The access to the leader election and metadata storage services */
     	private final HighAvailabilityServices haServices;
     
    -	// --------- resource manager --------
    +	/** The task manager configuration */
    +	private final TaskExecutorConfiguration taskExecutorConfig;
    +
    +	/** The connection information of the task manager */
    +	private final InstanceConnectionInfo connectionInfo;
    +
    +	/** The I/O manager component in the task manager */
    +	private final IOManager ioManager;
    +
    +	/** The memory manager component in the task manager */
    +	private final MemoryManager memoryManager;
    +
    +	/** The network component in the task manager */
    +	private final NetworkEnvironment networkEnvironment;
    +
    +	/** The number of slots in the task manager, should be 1 for YARN */
    +	private final int numberOfSlots;
     
    +	// --------- resource manager --------
     	private TaskExecutorToResourceManagerConnection resourceManagerConnection;
     
     	// ------------------------------------------------------------------------
     
     	public TaskExecutor(
    +			TaskExecutorConfiguration taskExecutorConfig,
    +			ResourceID resourceID,
    +			InstanceConnectionInfo connectionInfo,
    +			MemoryManager memoryManager,
    +			IOManager ioManager,
    +			NetworkEnvironment networkEnvironment,
    +			int numberOfSlots,
     			RpcService rpcService,
    -			HighAvailabilityServices haServices,
    -			ResourceID resourceID) {
    +			HighAvailabilityServices haServices) {
     
     		super(rpcService);
     
    -		this.haServices = checkNotNull(haServices);
    +		this.taskExecutorConfig = checkNotNull(taskExecutorConfig);
     		this.resourceID = checkNotNull(resourceID);
    +		this.connectionInfo = checkNotNull(connectionInfo);
    +		this.memoryManager = checkNotNull(memoryManager);
    +		this.ioManager = checkNotNull(ioManager);
    +		this.networkEnvironment = checkNotNull(networkEnvironment);
    +		this.numberOfSlots = checkNotNull(numberOfSlots);
    +		this.haServices = checkNotNull(haServices);
    +	}
    +
    +	/**
    +	 * Starts and runs the TaskManager.
    +	 * <p/>
    +	 * This method first tries to select the network interface to use for the TaskManager
    +	 * communication. The network interface is used both for the actor communication
    +	 * (coordination) as well as for the data exchange between task managers. Unless
    +	 * the hostname/interface is explicitly configured in the configuration, this
    +	 * method will try out various interfaces and methods to connect to the JobManager
    +	 * and select the one where the connection attempt is successful.
    +	 * <p/>
    +	 * After selecting the network interface, this method brings up an actor system
    +	 * for the TaskManager and its actors, starts the TaskManager's services
    +	 * (library cache, shuffle network stack, ...), and starts the TaskManager itself.
    +	 *
    +	 * @param configuration    The configuration for the TaskManager.
    +	 * @param resourceID       The id of the resource which the task manager will run on.
    +	 */
    +	public static void selectNetworkInterfaceAndRunTaskManager(
    +		Configuration configuration,
    +		ResourceID resourceID) throws Exception {
    +
    +		InetSocketAddress taskManagerAddress = selectNetworkInterfaceAndPort(configuration);
    +
    +		runTaskManager(taskManagerAddress.getHostName(), resourceID, taskManagerAddress.getPort(), configuration);
    +	}
    +
    +	private static InetSocketAddress selectNetworkInterfaceAndPort(Configuration configuration)
    +		throws Exception {
    +		String taskManagerHostname = configuration.getString(ConfigConstants.TASK_MANAGER_HOSTNAME_KEY, null);
    +		if (taskManagerHostname != null) {
    +			LOG.info("Using configured hostname/address for TaskManager: " + taskManagerHostname);
    +		} else {
    +			LeaderRetrievalService leaderRetrievalService = LeaderRetrievalUtils.createLeaderRetrievalService(configuration);
    +			FiniteDuration lookupTimeout = AkkaUtils.getLookupTimeout(configuration);
    +
    +			InetAddress taskManagerAddress = LeaderRetrievalUtils.findConnectingAddress(leaderRetrievalService, lookupTimeout);
    +			taskManagerHostname = taskManagerAddress.getHostName();
    +			LOG.info("TaskManager will use hostname/address '{}' ({}) for communication.",
    +				taskManagerHostname, taskManagerAddress.getHostAddress());
    +		}
    +
    +		// if no task manager port has been configured, use 0 (system will pick any free port)
    +		int actorSystemPort = configuration.getInteger(ConfigConstants.TASK_MANAGER_IPC_PORT_KEY, 0);
    +		if (actorSystemPort < 0 || actorSystemPort > 65535) {
    +			throw new IllegalConfigurationException("Invalid value for '" +
    +				ConfigConstants.TASK_MANAGER_IPC_PORT_KEY +
    +				"' (port for the TaskManager actor system) : " + actorSystemPort +
    +				" - Leave config parameter empty or use 0 to let the system choose a port automatically.");
    +		}
    +
    +		return new InetSocketAddress(taskManagerHostname, actorSystemPort);
    +	}
    +
    +	/**
    +	 * Starts and runs the TaskManager. Brings up an actor system for the TaskManager and its
    +	 * actors, starts the TaskManager's services (library cache, shuffle network stack, ...),
    +	 * and starts the TaskManager itself.
    +	 * <p/>
    +	 * This method will also spawn a process reaper for the TaskManager (kill the process if
    +	 * the actor fails) and optionally start the JVM memory logging thread.
    +	 *
    +	 * @param taskManagerHostname The hostname/address of the interface where the actor system
    +	 *                            will communicate.
    +	 * @param resourceID          The id of the resource which the task manager will run on.
    +	 * @param actorSystemPort   The port at which the actor system will communicate.
    +	 * @param configuration       The configuration for the TaskManager.
    +	 */
    +	private static void runTaskManager(
    +		String taskManagerHostname,
    +		ResourceID resourceID,
    +		int actorSystemPort,
    +		final Configuration configuration) throws Exception {
    +
    +		LOG.info("Starting TaskManager");
    +
    +		// Bring up the TaskManager actor system first, bind it to the given address.
    +
    +		LOG.info("Starting TaskManager actor system at " +
    +			NetUtils.hostAndPortToUrlString(taskManagerHostname, actorSystemPort));
    +
    +		final ActorSystem taskManagerSystem;
    +		try {
    +			Tuple2<String, Object> address = new Tuple2<String, Object>(taskManagerHostname, actorSystemPort);
    +			Config akkaConfig = AkkaUtils.getAkkaConfig(configuration, new Some<>(address));
    +			LOG.debug("Using akka configuration\n " + akkaConfig);
    +			taskManagerSystem = AkkaUtils.createActorSystem(akkaConfig);
    +		} catch (Throwable t) {
    +			if (t instanceof org.jboss.netty.channel.ChannelException) {
    +				Throwable cause = t.getCause();
    +				if (cause != null && t.getCause() instanceof java.net.BindException) {
    +					String address = NetUtils.hostAndPortToUrlString(taskManagerHostname, actorSystemPort);
    +					throw new IOException("Unable to bind TaskManager actor system to address " +
    +						address + " - " + cause.getMessage(), t);
    +				}
    +			}
    +			throw new Exception("Could not create TaskManager actor system", t);
    +		}
    +
    +		// start akka rpc service based on actor system
    +		Timeout timeout = new Timeout(AkkaUtils.getTimeout(configuration).toMillis(), TimeUnit.MILLISECONDS);
    +		AkkaRpcService akkaRpcService = new AkkaRpcService(taskManagerSystem, timeout);
    +
    +		// start high availability service to implement getResourceManagerLeaderRetriever method only
    +		HighAvailabilityServices haServices = new HighAvailabilityServices() {
    +			@Override
    +			public LeaderRetrievalService getResourceManagerLeaderRetriever() throws Exception {
    +				return LeaderRetrievalUtils.createLeaderRetrievalService(configuration);
    +			}
    +
    +			@Override
    +			public LeaderElectionService getJobMasterLeaderElectionService(JobID jobID) throws Exception {
    +				return null;
    +			}
    +		};
    +
    +		// start all the TaskManager services (network stack,  library cache, ...)
    +		// and the TaskManager actor
    +		try {
    +			LOG.info("Starting TaskManager actor");
    +			TaskExecutor taskExecutor = startTaskManagerComponentsAndActor(
    +				configuration,
    +				resourceID,
    +				akkaRpcService,
    +				taskManagerHostname,
    +				haServices,
    +				false);
    +
    +			taskExecutor.start();
    +
    +			// if desired, start the logging daemon that periodically logs the memory usage information
    +			if (LOG.isInfoEnabled() && configuration.getBoolean(
    +				ConfigConstants.TASK_MANAGER_DEBUG_MEMORY_USAGE_START_LOG_THREAD,
    +				ConfigConstants.DEFAULT_TASK_MANAGER_DEBUG_MEMORY_USAGE_START_LOG_THREAD)) {
    +				LOG.info("Starting periodic memory usage logger");
    +
    +				long interval = configuration.getLong(
    +					ConfigConstants.TASK_MANAGER_DEBUG_MEMORY_USAGE_LOG_INTERVAL_MS,
    +					ConfigConstants.DEFAULT_TASK_MANAGER_DEBUG_MEMORY_USAGE_LOG_INTERVAL_MS);
    +
    +				MemoryLogger logger = new MemoryLogger(LOG, interval, taskManagerSystem);
    +				logger.start();
    +			}
    +
    +			// block until everything is done
    +			taskManagerSystem.awaitTermination();
    +		} catch (Throwable t) {
    +			LOG.error("Error while starting up taskManager", t);
    +			try {
    +				taskManagerSystem.shutdown();
    +			} catch (Throwable tt) {
    +				LOG.warn("Could not cleanly shut down actor system", tt);
    +			}
    +			throw t;
    +		}
    +	}
    +
    +	// --------------------------------------------------------------------------
    +	//  Starting and running the TaskManager
    +	// --------------------------------------------------------------------------
    +
    +	/**
    +	 * @param configuration                 The configuration for the TaskManager.
    +	 * @param resourceID                    The id of the resource which the task manager will run on.
    +	 * @param rpcService                  The rpc service which is used to start and connect to the TaskManager RpcEndpoint .
    +	 * @param taskManagerHostname       The hostname/address that describes the TaskManager's data location.
    +	 * @param haServices        Optionally, a high availability service can be provided. If none is given,
    +	 *                                      then a HighAvailabilityServices is constructed from the configuration.
    +	 * @param localTaskManagerCommunication If true, the TaskManager will not initiate the TCP network stack.
    +	 * @return An ActorRef to the TaskManager actor.
    +	 * @throws org.apache.flink.configuration.IllegalConfigurationException     Thrown, if the given config contains illegal values.
    +	 * @throws java.io.IOException      Thrown, if any of the I/O components (such as buffer pools,
    +	 *                                       I/O manager, ...) cannot be properly started.
    +	 * @throws java.lang.Exception      Thrown is some other error occurs while parsing the configuration
    +	 *                                      or starting the TaskManager components.
    +	 */
    +	public static TaskExecutor startTaskManagerComponentsAndActor(
    +		Configuration configuration,
    +		ResourceID resourceID,
    +		RpcService rpcService,
    +		String taskManagerHostname,
    +		HighAvailabilityServices haServices,
    +		boolean localTaskManagerCommunication) throws Exception {
    +
    +		Tuple4<TaskExecutorConfiguration, NetworkEnvironmentConfiguration, InstanceConnectionInfo, MemoryType> tuple4
    +			= parseTaskManagerConfiguration(configuration, taskManagerHostname, localTaskManagerCommunication);
    +
    +		TaskExecutorConfiguration taskExecutorConfig = tuple4.f0;
    +		NetworkEnvironmentConfiguration netConfig = tuple4.f1;
    +		InstanceConnectionInfo connectionInfo = tuple4.f2;
    +		MemoryType memType = tuple4.f3;
    +
    +		// pre-start checks
    +		checkTempDirs(taskExecutorConfig.getTmpDirPaths());
    +
    +		ExecutionContext executionContext = ExecutionContexts$.MODULE$.fromExecutor(new ForkJoinPool());
    +
    +		// we start the network first, to make sure it can allocate its buffers first
    +		NetworkEnvironment network = new NetworkEnvironment(executionContext, taskExecutorConfig.getTimeout(), netConfig);
    +
    +		// computing the amount of memory to use depends on how much memory is available
    +		// it strictly needs to happen AFTER the network stack has been initialized
    +
    +		// check if a value has been configured
    +		long configuredMemory = configuration.getLong(ConfigConstants.TASK_MANAGER_MEMORY_SIZE_KEY, -1L);
    +		checkConfigParameter(configuredMemory == -1 || configuredMemory > 0, configuredMemory,
    +			ConfigConstants.TASK_MANAGER_MEMORY_SIZE_KEY,
    +			"MemoryManager needs at least one MB of memory. " +
    +				"If you leave this config parameter empty, the system automatically " +
    +				"pick a fraction of the available memory.");
    +
    +		long memorySize;
    +		boolean preAllocateMemory = configuration.getBoolean(
    +			ConfigConstants.TASK_MANAGER_MEMORY_PRE_ALLOCATE_KEY,
    +			ConfigConstants.DEFAULT_TASK_MANAGER_MEMORY_PRE_ALLOCATE);
    +		if (configuredMemory > 0) {
    +			if (preAllocateMemory) {
    +				LOG.info("Using {} MB for managed memory." , configuredMemory);
    +			} else {
    +				LOG.info("Limiting managed memory to {} MB, memory will be allocated lazily." , configuredMemory);
    +			}
    +			memorySize = configuredMemory << 20; // megabytes to bytes
    +		} else {
    +			float fraction = configuration.getFloat(
    +				ConfigConstants.TASK_MANAGER_MEMORY_FRACTION_KEY,
    +				ConfigConstants.DEFAULT_MEMORY_MANAGER_MEMORY_FRACTION);
    +			checkConfigParameter(fraction > 0.0f && fraction < 1.0f, fraction,
    +				ConfigConstants.TASK_MANAGER_MEMORY_FRACTION_KEY,
    +				"MemoryManager fraction of the free memory must be between 0.0 and 1.0");
    +
    +			if (memType == MemoryType.HEAP) {
    +				long relativeMemSize = (long) (EnvironmentInformation.getSizeOfFreeHeapMemoryWithDefrag() * fraction);
    +				if (preAllocateMemory) {
    +					LOG.info("Using {} of the currently free heap space for managed heap memory ({} MB)." ,
    +						fraction , relativeMemSize >> 20);
    +				} else {
    +					LOG.info("Limiting managed memory to {} of the currently free heap space ({} MB), " +
    +						"memory will be allocated lazily." , fraction , relativeMemSize >> 20);
    +				}
    +				memorySize = relativeMemSize;
    +			} else if (memType == MemoryType.OFF_HEAP) {
    +				// The maximum heap memory has been adjusted according to the fraction
    +				long maxMemory = EnvironmentInformation.getMaxJvmHeapMemory();
    +				long directMemorySize = (long) (maxMemory / (1.0 - fraction) * fraction);
    +				if (preAllocateMemory) {
    +					LOG.info("Using {} of the maximum memory size for managed off-heap memory ({} MB)." ,
    +						fraction, directMemorySize >> 20);
    +				} else {
    +					LOG.info("Limiting managed memory to {} of the maximum memory size ({} MB)," +
    +						" memory will be allocated lazily.", fraction, directMemorySize >> 20);
    +				}
    +				memorySize = directMemorySize;
    +			} else {
    +				throw new RuntimeException("No supported memory type detected.");
    +			}
    +		}
    +
    +		// now start the memory manager
    +		final MemoryManager memoryManager;
    +		try {
    +			memoryManager = new MemoryManager(
    +				memorySize,
    +				taskExecutorConfig.getNumberOfSlots(),
    +				netConfig.networkBufferSize(),
    +				memType,
    +				preAllocateMemory);
    +		} catch (OutOfMemoryError e) {
    +			if (memType == MemoryType.HEAP) {
    +				throw new Exception("OutOfMemory error (" + e.getMessage() +
    +					") while allocating the TaskManager heap memory (" + memorySize + " bytes).", e);
    +			} else if (memType == MemoryType.OFF_HEAP) {
    +				throw new Exception("OutOfMemory error (" + e.getMessage() +
    +					") while allocating the TaskManager off-heap memory (" + memorySize +
    +					" bytes).Try increasing the maximum direct memory (-XX:MaxDirectMemorySize)", e);
    +			} else {
    +				throw e;
    +			}
    +		}
    +
    +		// start the I/O manager, it will create some temp directories.
    +		IOManager ioManager = new IOManagerAsync(taskExecutorConfig.getTmpDirPaths());
    +
    +		TaskExecutor taskExecutor = new TaskExecutor(
    +			taskExecutorConfig,
    +			resourceID,
    +			connectionInfo,
    +			memoryManager,
    +			ioManager,
    +			network,
    +			taskExecutorConfig.getNumberOfSlots(),
    +			rpcService,
    +			haServices);
    +
    +		return taskExecutor;
    +	}
    +
    +	// --------------------------------------------------------------------------
    +	//  Parsing and checking the TaskManager Configuration
    +	// --------------------------------------------------------------------------
    +
    +	/**
    +	 * Utility method to extract TaskManager config parameters from the configuration and to
    +	 * sanity check them.
    +	 *
    +	 * @param configuration                 The configuration.
    +	 * @param taskManagerHostname           The host name under which the TaskManager communicates.
    +	 * @param localTaskManagerCommunication True, to skip initializing the network stack.
    +	 *                                      Use only in cases where only one task manager runs.
    +	 * @return A tuple (TaskManagerConfiguration, network configuration,
    +	 * InstanceConnectionInfo, JobManager actor Akka URL).
    +	 */
    +	private static Tuple4<TaskExecutorConfiguration, NetworkEnvironmentConfiguration, InstanceConnectionInfo, MemoryType>
    --- End diff --
    
    Also you could consider moving these methods into a factory which you pass to the TaskExecutor.


---
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.
---

[GitHub] flink pull request #2400: [FLINK-4363] Implement TaskManager basic startup o...

Posted by mxm <gi...@git.apache.org>.
Github user mxm commented on a diff in the pull request:

    https://github.com/apache/flink/pull/2400#discussion_r75648837
  
    --- Diff: flink-runtime/src/main/java/org/apache/flink/runtime/rpc/taskexecutor/TaskExecutor.java ---
    @@ -35,27 +79,634 @@
      */
     public class TaskExecutor extends RpcEndpoint<TaskExecutorGateway> {
     
    -	/** The unique resource ID of this TaskExecutor */
    +	private static final Logger LOG = LoggerFactory.getLogger(TaskExecutor.class);
    +
    +	/** Return code for critical errors during the runtime */
    +	private static final int RUNTIME_FAILURE_RETURN_CODE = 2;
    +
    +	/** The name of the TaskManager actor */
    +	private static final String TASK_MANAGER_NAME = "taskmanager";
    --- End diff --
    
    This is still named "taskmanager"


---
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.
---

[GitHub] flink issue #2400: [FLINK-4363] Implement TaskManager basic startup of all c...

Posted by mxm <gi...@git.apache.org>.
Github user mxm commented on the issue:

    https://github.com/apache/flink/pull/2400
  
    I think the hostname parameter is required for some testing code which explicitly uses "localhost" as hostname.


---
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.
---

[GitHub] flink issue #2400: [FLINK-4363] Implement TaskManager basic startup of all c...

Posted by wangzhijiang999 <gi...@git.apache.org>.
Github user wangzhijiang999 commented on the issue:

    https://github.com/apache/flink/pull/2400
  
    Thank you for your review and advice, mxm. Another concern is whether to modify the parameters in the methods "selectNetworkInterfaceAndRunTaskManager" and "startTaskManagerComponentsAndActor" which will be invoked by "YarnTMRunner" and "LocalFlinkMiniCluster". The main differences for new TM are "RPCService" and "HighAvailabilityServices" instead of "ActorSystem" and "LeaderRetrievalService". My current implementation is only supporting these new components in constructor for easy test and retain the previous methods and parameters.  Should we further modify the related parts with "ActorSystem" and "LeaderRetrievalService" next or just like this currently?


---
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.
---

[GitHub] flink issue #2400: [FLINK-4363] Implement TaskManager basic startup of all c...

Posted by mxm <gi...@git.apache.org>.
Github user mxm commented on the issue:

    https://github.com/apache/flink/pull/2400
  
    Could you please remove the merge commits from this PR? We don't use merge commits in the Flink repository to have an easy to read code history. Thank you.
    
    Thanks for opening FLINK-4505. For now, let us focus on implementing the factory to bring up the TaskExecutor for testing with the FlinkMiniCluster. I think it would be nice to remove as much startup code from the TaskExecutor class as possible. Any code which can be shared among all `TaskExecutorFactory`s can be put in the abstract `TaskExecutorFactoryBase` class.


---
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.
---

[GitHub] flink issue #2400: [FLINK-4363] Implement TaskManager basic startup of all c...

Posted by wangzhijiang999 <gi...@git.apache.org>.
Github user wangzhijiang999 commented on the issue:

    https://github.com/apache/flink/pull/2400
  
    @mxm , could we remove the parameter "taskManagerHostname" from "startTaskManagerComponentsAndActor" method, I think the hostname can be got by {{TaskExecutor}} itself, no need to pass it from outsider world. To do so, it is more easier to construct {{TaskExecutor}} by factory method "startTaskManagerComponentsAndActor(configuration, resourceID,rpcService,haServices,localTaskManagerCommunication)" when design UT for testing.



---
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.
---

[GitHub] flink pull request #2400: [FLINK-4363] Implement TaskManager basic startup o...

Posted by mxm <gi...@git.apache.org>.
Github user mxm commented on a diff in the pull request:

    https://github.com/apache/flink/pull/2400#discussion_r75647937
  
    --- Diff: flink-runtime/src/main/java/org/apache/flink/runtime/rpc/taskexecutor/TaskExecutor.java ---
    @@ -35,27 +79,634 @@
      */
     public class TaskExecutor extends RpcEndpoint<TaskExecutorGateway> {
     
    -	/** The unique resource ID of this TaskExecutor */
    +	private static final Logger LOG = LoggerFactory.getLogger(TaskExecutor.class);
    +
    +	/** Return code for critical errors during the runtime */
    +	private static final int RUNTIME_FAILURE_RETURN_CODE = 2;
    +
    +	/** The name of the TaskManager actor */
    +	private static final String TASK_MANAGER_NAME = "taskmanager";
    +
    +	/** The unique resource ID of this TaskManager */
     	private final ResourceID resourceID;
     
     	/** The access to the leader election and metadata storage services */
     	private final HighAvailabilityServices haServices;
     
    -	// --------- resource manager --------
    +	/** The task manager configuration */
    +	private final TaskManagerConfiguration taskManagerConfig;
    +
    +	/** The connection information of the task manager */
    +	private final InstanceConnectionInfo connectionInfo;
    +
    +	/** The I/O manager component in the task manager */
    +	private final IOManager ioManager;
    +
    +	/** The memory manager component in the task manager */
    +	private final MemoryManager memoryManager;
    +
    +	/** The network component in the task manager */
    +	private final NetworkEnvironment networkEnvironment;
    +
    +	/** The number of slots in the task manager, should be 1 for YARN */
    +	private final int numberOfSlots;
     
    +	// --------- resource manager --------
     	private TaskExecutorToResourceManagerConnection resourceManagerConnection;
     
     	// ------------------------------------------------------------------------
     
     	public TaskExecutor(
    +			TaskManagerConfiguration taskManagerConfig,
    +			ResourceID resourceID,
    +			InstanceConnectionInfo connectionInfo,
    +			MemoryManager memoryManager,
    +			IOManager ioManager,
    +			NetworkEnvironment networkEnvironment,
    +			int numberOfSlots,
     			RpcService rpcService,
    -			HighAvailabilityServices haServices,
    -			ResourceID resourceID) {
    +			HighAvailabilityServices haServices) {
     
     		super(rpcService);
     
    -		this.haServices = checkNotNull(haServices);
    +		this.taskManagerConfig = checkNotNull(taskManagerConfig);
     		this.resourceID = checkNotNull(resourceID);
    +		this.connectionInfo = checkNotNull(connectionInfo);
    +		this.memoryManager = checkNotNull(memoryManager);
    +		this.ioManager = checkNotNull(ioManager);
    +		this.networkEnvironment = checkNotNull(networkEnvironment);
    +		this.numberOfSlots = checkNotNull(numberOfSlots);
    +		this.haServices = checkNotNull(haServices);
    +	}
    +
    +	/**
    +	 * Starts and runs the TaskManager.
    +	 * <p/>
    +	 * This method first tries to select the network interface to use for the TaskManager
    +	 * communication. The network interface is used both for the actor communication
    +	 * (coordination) as well as for the data exchange between task managers. Unless
    +	 * the hostname/interface is explicitly configured in the configuration, this
    +	 * method will try out various interfaces and methods to connect to the JobManager
    +	 * and select the one where the connection attempt is successful.
    +	 * <p/>
    +	 * After selecting the network interface, this method brings up an actor system
    +	 * for the TaskManager and its actors, starts the TaskManager's services
    +	 * (library cache, shuffle network stack, ...), and starts the TaskManager itself.
    +	 *
    +	 * @param configuration    The configuration for the TaskManager.
    +	 * @param taskManagerClass The actor class to instantiate.
    +	 *                         Allows to use TaskManager subclasses for example for YARN.
    +	 */
    +	public static void selectNetworkInterfaceAndRunTaskManager(
    +		Configuration configuration,
    +		ResourceID resourceID,
    +		Class<? extends TaskManager> taskManagerClass) throws Exception {
    +
    +		Tuple2<String, Integer> tuple2 = selectNetworkInterfaceAndPort(configuration);
    +
    +		runTaskManager(tuple2._1(), resourceID, tuple2._2(), configuration, taskManagerClass);
    +	}
    +
    +	private static Tuple2<String, Integer> selectNetworkInterfaceAndPort(Configuration configuration)
    +		throws Exception {
    +		String taskManagerHostname = configuration.getString(ConfigConstants.TASK_MANAGER_HOSTNAME_KEY, null);
    +		if (taskManagerHostname != null) {
    +			LOG.info("Using configured hostname/address for TaskManager: " + taskManagerHostname);
    +		} else {
    +			LeaderRetrievalService leaderRetrievalService = LeaderRetrievalUtils.createLeaderRetrievalService(configuration);
    +			FiniteDuration lookupTimeout = AkkaUtils.getLookupTimeout(configuration);
    +
    +			InetAddress taskManagerAddress = LeaderRetrievalUtils.findConnectingAddress(leaderRetrievalService, lookupTimeout);
    +			taskManagerHostname = taskManagerAddress.getHostName();
    +			LOG.info("TaskManager will use hostname/address '{}' ({}) for communication.",
    +				taskManagerHostname, taskManagerAddress.getHostAddress());
    +		}
    +
    +		// if no task manager port has been configured, use 0 (system will pick any free port)
    +		int actorSystemPort = configuration.getInteger(ConfigConstants.TASK_MANAGER_IPC_PORT_KEY, 0);
    +		if (actorSystemPort < 0 || actorSystemPort > 65535) {
    +			throw new IllegalConfigurationException("Invalid value for '" +
    +				ConfigConstants.TASK_MANAGER_IPC_PORT_KEY +
    +				"' (port for the TaskManager actor system) : " + actorSystemPort +
    +				" - Leave config parameter empty or use 0 to let the system choose a port automatically.");
    +		}
    +
    +		return new Tuple2<>(taskManagerHostname, actorSystemPort);
    +	}
    +
    +	/**
    +	 * Starts and runs the TaskManager. Brings up an actor system for the TaskManager and its
    +	 * actors, starts the TaskManager's services (library cache, shuffle network stack, ...),
    +	 * and starts the TaskManager itself.
    +	 * <p/>
    +	 * This method will also spawn a process reaper for the TaskManager (kill the process if
    +	 * the actor fails) and optionally start the JVM memory logging thread.
    +	 *
    +	 * @param taskManagerHostname The hostname/address of the interface where the actor system
    +	 *                            will communicate.
    +	 * @param resourceID          The id of the resource which the task manager will run on.
    +	 * @param actorSystemPort     The port at which the actor system will communicate.
    +	 * @param configuration       The configuration for the TaskManager.
    +	 * @param taskManagerClass    The actor class to instantiate. Allows the use of TaskManager
    +	 *                            subclasses for example for YARN.
    +	 */
    +	private static void runTaskManager(
    +		String taskManagerHostname,
    +		ResourceID resourceID,
    +		int actorSystemPort,
    +		Configuration configuration,
    +		Class<? extends TaskManager> taskManagerClass) throws Exception {
    +
    +		LOG.info("Starting TaskManager");
    +
    +		// Bring up the TaskManager actor system first, bind it to the given address.
    +
    +		LOG.info("Starting TaskManager actor system at " +
    +			NetUtils.hostAndPortToUrlString(taskManagerHostname, actorSystemPort));
    +
    +		ActorSystem taskManagerSystem;
    +		try {
    +			Tuple2<String, Object> address = new Tuple2<String, Object>(taskManagerHostname, actorSystemPort);
    +			Config akkaConfig = AkkaUtils.getAkkaConfig(configuration, new Some<>(address));
    +			LOG.debug("Using akka configuration\n " + akkaConfig);
    +			taskManagerSystem = AkkaUtils.createActorSystem(akkaConfig);
    +		} catch (Throwable t) {
    +			if (t instanceof org.jboss.netty.channel.ChannelException) {
    +				Throwable cause = t.getCause();
    +				if (cause != null && t.getCause() instanceof java.net.BindException) {
    +					String address = NetUtils.hostAndPortToUrlString(taskManagerHostname, actorSystemPort);
    +					throw new IOException("Unable to bind TaskManager actor system to address " +
    +						address + " - " + cause.getMessage(), t);
    +				}
    +			}
    +			throw new Exception("Could not create TaskManager actor system", t);
    +		}
    +
    +		// start all the TaskManager services (network stack,  library cache, ...)
    +		// and the TaskManager actor
    +		try {
    +			LOG.info("Starting TaskManager actor");
    +			ActorRef taskManagerActor = startTaskManagerComponentsAndActor(
    +				configuration,
    +				resourceID,
    +				taskManagerSystem,
    +				taskManagerHostname,
    +				TASK_MANAGER_NAME,
    +				null,
    +				false,
    +				taskManagerClass);
    +
    +			// start a process reaper that watches the JobManager. If the TaskManager actor dies,
    +			// the process reaper will kill the JVM process (to ensure easy failure detection)
    +			LOG.debug("Starting TaskManager process reaper");
    +
    +			taskManagerSystem.actorOf(
    +				Props.create(ProcessReaper.class, taskManagerActor, LOG, RUNTIME_FAILURE_RETURN_CODE),
    +				"TaskManager_Process_Reaper");
    +
    +			// if desired, start the logging daemon that periodically logs the memory usage information
    +			if (LOG.isInfoEnabled() && configuration.getBoolean(
    +				ConfigConstants.TASK_MANAGER_DEBUG_MEMORY_USAGE_START_LOG_THREAD,
    +				ConfigConstants.DEFAULT_TASK_MANAGER_DEBUG_MEMORY_USAGE_START_LOG_THREAD)) {
    +				LOG.info("Starting periodic memory usage logger");
    +
    +				long interval = configuration.getLong(
    +					ConfigConstants.TASK_MANAGER_DEBUG_MEMORY_USAGE_LOG_INTERVAL_MS,
    +					ConfigConstants.DEFAULT_TASK_MANAGER_DEBUG_MEMORY_USAGE_LOG_INTERVAL_MS);
    +
    +				MemoryLogger logger = new MemoryLogger(LOG, interval, taskManagerSystem);
    +				logger.start();
    +			}
    +
    +			// block until everything is done
    +			taskManagerSystem.awaitTermination();
    +		} catch (Throwable t) {
    +			LOG.error("Error while starting up taskManager", t);
    +			try {
    +				taskManagerSystem.shutdown();
    +			} catch (Throwable tt) {
    +				LOG.warn("Could not cleanly shut down actor system", tt);
    +			}
    +			throw t;
    +		}
    +	}
    +
    +	// --------------------------------------------------------------------------
    +	//  Starting and running the TaskManager
    +	// --------------------------------------------------------------------------
    +
    +	/**
    +	 * @param configuration                 The configuration for the TaskManager.
    +	 * @param resourceID                    The id of the resource which the task manager will run on.
    +	 * @param actorSystem                  The actor system that should run the TaskManager actor.
    +	 * @param taskManagerHostname       The hostname/address that describes the TaskManager's data location.
    +	 * @param taskManagerActorName      Optionally the name of the TaskManager actor. If none is given,
    +	 *                                      the actor will use a random name.
    +	 * @param leaderRetrievalService        Optionally, a leader retrieval service can be provided. If none is given,
    +	 *                                      then a LeaderRetrievalService is constructed from the configuration.
    +	 * @param localTaskManagerCommunication If true, the TaskManager will not initiate the TCP network stack.
    +	 * @param taskManagerClass      The class of the TaskManager actor. May be used to give
    +	 *                                      subclasses that understand additional actor messages.
    +	 * @return An ActorRef to the TaskManager actor.
    +	 * @throws org.apache.flink.configuration.IllegalConfigurationException     Thrown, if the given config contains illegal values.
    +	 * @throws java.io.IOException      Thrown, if any of the I/O components (such as buffer pools,
    +	 *                                       I/O manager, ...) cannot be properly started.
    +	 * @throws java.lang.Exception      Thrown is some other error occurs while parsing the configuration
    +	 *                                      or starting the TaskManager components.
    +	 */
    +	public static ActorRef startTaskManagerComponentsAndActor(
    +		Configuration configuration,
    +		ResourceID resourceID,
    +		ActorSystem actorSystem,
    +		String taskManagerHostname,
    +		String taskManagerActorName,
    +		LeaderRetrievalService leaderRetrievalService,
    +		boolean localTaskManagerCommunication,
    +		Class<? extends TaskManager> taskManagerClass) throws Exception {
    +
    +		Tuple4<TaskManagerConfiguration, NetworkEnvironmentConfiguration, InstanceConnectionInfo, MemoryType> tuple4
    +			= parseTaskManagerConfiguration(configuration, taskManagerHostname, localTaskManagerCommunication);
    +
    +		TaskManagerConfiguration taskManagerConfig = tuple4._1();
    +		NetworkEnvironmentConfiguration netConfig = tuple4._2();
    +		InstanceConnectionInfo connectionInfo = tuple4._3();
    +		MemoryType memType = tuple4._4();
    +
    +		// pre-start checks
    +		checkTempDirs(taskManagerConfig.tmpDirPaths());
    +
    +		ExecutionContext executionContext = ExecutionContexts$.MODULE$.fromExecutor(new ForkJoinPool());
    +
    +		// we start the network first, to make sure it can allocate its buffers first
    +		NetworkEnvironment network = new NetworkEnvironment(executionContext, taskManagerConfig.timeout(), netConfig);
    +
    +		// computing the amount of memory to use depends on how much memory is available
    +		// it strictly needs to happen AFTER the network stack has been initialized
    +
    +		// check if a value has been configured
    +		long configuredMemory = configuration.getLong(ConfigConstants.TASK_MANAGER_MEMORY_SIZE_KEY, -1L);
    +		checkConfigParameter(configuredMemory == -1 || configuredMemory > 0, configuredMemory,
    +			ConfigConstants.TASK_MANAGER_MEMORY_SIZE_KEY,
    +			"MemoryManager needs at least one MB of memory. " +
    +				"If you leave this config parameter empty, the system automatically " +
    +				"pick a fraction of the available memory.");
    +
    +		long memorySize;
    +		boolean preAllocateMemory = configuration.getBoolean(
    +			ConfigConstants.TASK_MANAGER_MEMORY_PRE_ALLOCATE_KEY,
    +			ConfigConstants.DEFAULT_TASK_MANAGER_MEMORY_PRE_ALLOCATE);
    +		if (configuredMemory > 0) {
    +			if (preAllocateMemory) {
    +				LOG.info("Using {} MB for managed memory." , configuredMemory);
    +			} else {
    +				LOG.info("Limiting managed memory to {} MB, memory will be allocated lazily." , configuredMemory);
    +			}
    +			memorySize = configuredMemory << 20; // megabytes to bytes
    +		} else {
    +			float fraction = configuration.getFloat(
    +				ConfigConstants.TASK_MANAGER_MEMORY_FRACTION_KEY,
    +				ConfigConstants.DEFAULT_MEMORY_MANAGER_MEMORY_FRACTION);
    +			checkConfigParameter(fraction > 0.0f && fraction < 1.0f, fraction,
    +				ConfigConstants.TASK_MANAGER_MEMORY_FRACTION_KEY,
    +				"MemoryManager fraction of the free memory must be between 0.0 and 1.0");
    +
    +			if (memType == MemoryType.HEAP) {
    +				long relativeMemSize = (long) (EnvironmentInformation.getSizeOfFreeHeapMemoryWithDefrag() * fraction);
    +				if (preAllocateMemory) {
    +					LOG.info("Using {} of the currently free heap space for managed heap memory ({} MB)." ,
    +						fraction , relativeMemSize >> 20);
    +				} else {
    +					LOG.info("Limiting managed memory to {} of the currently free heap space ({} MB), " +
    +						"memory will be allocated lazily." , fraction , relativeMemSize >> 20);
    +				}
    +				memorySize = relativeMemSize;
    +			} else if (memType == MemoryType.OFF_HEAP) {
    +				// The maximum heap memory has been adjusted according to the fraction
    +				long maxMemory = EnvironmentInformation.getMaxJvmHeapMemory();
    +				long directMemorySize = (long) (maxMemory / (1.0 - fraction) * fraction);
    +				if (preAllocateMemory) {
    +					LOG.info("Using {} of the maximum memory size for managed off-heap memory ({} MB)." ,
    +						fraction, directMemorySize >> 20);
    +				} else {
    +					LOG.info("Limiting managed memory to {} of the maximum memory size ({} MB)," +
    +						" memory will be allocated lazily.", fraction, directMemorySize >> 20);
    +				}
    +				memorySize = directMemorySize;
    +			} else {
    +				throw new RuntimeException("No supported memory type detected.");
    +			}
    +		}
    +
    +		// now start the memory manager
    +		MemoryManager memoryManager;
    +		try {
    +			memoryManager = new MemoryManager(
    +				memorySize,
    +				taskManagerConfig.numberOfSlots(),
    +				netConfig.networkBufferSize(),
    +				memType,
    +				preAllocateMemory);
    +		} catch (OutOfMemoryError e) {
    +			if (memType == MemoryType.HEAP) {
    +				throw new Exception("OutOfMemory error (" + e.getMessage() +
    +					") while allocating the TaskManager heap memory (" + memorySize + " bytes).", e);
    +			} else if (memType == MemoryType.OFF_HEAP) {
    +				throw new Exception("OutOfMemory error (" + e.getMessage() +
    +					") while allocating the TaskManager off-heap memory (" + memorySize +
    +					" bytes).Try increasing the maximum direct memory (-XX:MaxDirectMemorySize)", e);
    +			} else {
    +				throw e;
    +			}
    +		}
    +
    +		// start the I/O manager, it will create some temp directories.
    +		IOManager ioManager = new IOManagerAsync(taskManagerConfig.tmpDirPaths());
    +
    +		if (leaderRetrievalService == null){
    +			leaderRetrievalService = LeaderRetrievalUtils.createLeaderRetrievalService(configuration);
    +		}
    +
    +		// create the actor properties (which define the actor constructor parameters)
    +		Props tmProps = Props.create(
    +			taskManagerClass,
    +			taskManagerConfig,
    +			resourceID,
    +			connectionInfo,
    +			memoryManager,
    +			ioManager,
    +			network,
    +			taskManagerConfig.numberOfSlots(),
    +			leaderRetrievalService);
    +
    +		ActorRef taskManagerActorRef;
    +		if (taskManagerActorName != null && !taskManagerActorName.equals("")) {
    +			taskManagerActorRef = actorSystem.actorOf(tmProps, taskManagerActorName);
    +		} else {
    +			taskManagerActorRef = actorSystem.actorOf(tmProps);
    +		}
    +
    +		return taskManagerActorRef;
    +	}
    +
    +	// --------------------------------------------------------------------------
    +	//  Parsing and checking the TaskManager Configuration
    +	// --------------------------------------------------------------------------
    +
    +	/**
    +	 * Utility method to extract TaskManager config parameters from the configuration and to
    +	 * sanity check them.
    +	 *
    +	 * @param configuration                 The configuration.
    +	 * @param taskManagerHostname           The host name under which the TaskManager communicates.
    +	 * @param localTaskManagerCommunication True, to skip initializing the network stack.
    +	 *                                      Use only in cases where only one task manager runs.
    +	 * @return A tuple (TaskManagerConfiguration, network configuration,
    +	 * InstanceConnectionInfo, JobManager actor Akka URL).
    +	 */
    +	private static Tuple4<TaskManagerConfiguration, NetworkEnvironmentConfiguration, InstanceConnectionInfo, MemoryType>
    +		parseTaskManagerConfiguration(Configuration configuration, String taskManagerHostname, boolean localTaskManagerCommunication)
    +		throws Exception {
    +
    +		// ------- read values from the config and check them ---------
    +		//                      (a lot of them)
    +
    +		// ----> hosts / ports for communication and data exchange
    +
    +		int dataport = configuration.getInteger(ConfigConstants.TASK_MANAGER_DATA_PORT_KEY,
    +			ConfigConstants.DEFAULT_TASK_MANAGER_DATA_PORT);
    +		if (dataport == 0) {
    +			dataport = NetUtils.getAvailablePort();
    +		}
    +		checkConfigParameter(dataport > 0, dataport, ConfigConstants.TASK_MANAGER_DATA_PORT_KEY,
    +			"Leave config parameter empty or use 0 to let the system choose a port automatically.");
    +
    +		InetAddress taskManagerAddress = InetAddress.getByName(taskManagerHostname);
    +		InstanceConnectionInfo connectionInfo = new InstanceConnectionInfo(taskManagerAddress, dataport);
    +
    +		// ----> memory / network stack (shuffles/broadcasts), task slots, temp directories
    +
    +		// we need this because many configs have been written with a "-1" entry
    +		int slots = configuration.getInteger(ConfigConstants.TASK_MANAGER_NUM_TASK_SLOTS, 1);
    +		if (slots == -1) {
    +			slots = 1;
    +		}
    +		checkConfigParameter(slots >= 1, slots, ConfigConstants.TASK_MANAGER_NUM_TASK_SLOTS,
    +			"Number of task slots must be at least one.");
    +
    +		int numNetworkBuffers = configuration.getInteger(
    +			ConfigConstants.TASK_MANAGER_NETWORK_NUM_BUFFERS_KEY,
    +			ConfigConstants.DEFAULT_TASK_MANAGER_NETWORK_NUM_BUFFERS);
    +		checkConfigParameter(numNetworkBuffers > 0, numNetworkBuffers,
    +			ConfigConstants.TASK_MANAGER_NETWORK_NUM_BUFFERS_KEY, "");
    +
    +		int pageSize = configuration.getInteger(
    +			ConfigConstants.TASK_MANAGER_MEMORY_SEGMENT_SIZE_KEY,
    +			ConfigConstants.DEFAULT_TASK_MANAGER_MEMORY_SEGMENT_SIZE);
    +		// check page size of for minimum size
    +		checkConfigParameter(pageSize >= MemoryManager.MIN_PAGE_SIZE, pageSize,
    +			ConfigConstants.TASK_MANAGER_MEMORY_SEGMENT_SIZE_KEY,
    +			"Minimum memory segment size is " + MemoryManager.MIN_PAGE_SIZE);
    +		// check page size for power of two
    +		checkConfigParameter(MathUtils.isPowerOf2(pageSize), pageSize,
    +			ConfigConstants.TASK_MANAGER_MEMORY_SEGMENT_SIZE_KEY,
    +			"Memory segment size must be a power of 2.");
    +
    +		// check whether we use heap or off-heap memory
    +		MemoryType memType;
    +		if (configuration.getBoolean(ConfigConstants.TASK_MANAGER_MEMORY_OFF_HEAP_KEY, false)) {
    +			memType = MemoryType.OFF_HEAP;
    +		} else {
    +			memType = MemoryType.HEAP;
    +		}
    +
    +		// initialize the memory segment factory accordingly
    +		if (memType == MemoryType.HEAP) {
    +			if (!MemorySegmentFactory.isInitialized()) {
    +				MemorySegmentFactory.initializeFactory(HeapMemorySegment.FACTORY);
    +			} else if (MemorySegmentFactory.getFactory() != HeapMemorySegment.FACTORY) {
    +				throw new Exception("Memory type is set to heap memory, but memory segment " +
    +					"factory has been initialized for off-heap memory segments");
    +			}
    +		} else {
    +			if (!MemorySegmentFactory.isInitialized()) {
    +				MemorySegmentFactory.initializeFactory(HybridMemorySegment.FACTORY);
    +			} else if (MemorySegmentFactory.getFactory() != HybridMemorySegment.FACTORY) {
    +				throw new Exception("Memory type is set to off-heap memory, but memory segment " +
    +					"factory has been initialized for heap memory segments");
    +			}
    +		}
    +
    +		String[] tmpDirs = configuration.getString(
    +			ConfigConstants.TASK_MANAGER_TMP_DIR_KEY,
    +			ConfigConstants.DEFAULT_TASK_MANAGER_TMP_PATH).split(",|" + File.pathSeparator);
    +
    +		NettyConfig nettyConfig = null;
    +		if (!localTaskManagerCommunication) {
    +			nettyConfig = new NettyConfig(connectionInfo.address(), connectionInfo.dataPort(), pageSize, slots, configuration);
    +		}
    +
    +		// Default spill I/O mode for intermediate results
    +		String syncOrAsync = configuration.getString(
    +			ConfigConstants.TASK_MANAGER_NETWORK_DEFAULT_IO_MODE,
    +			ConfigConstants.DEFAULT_TASK_MANAGER_NETWORK_DEFAULT_IO_MODE);
    +
    +		IOMode ioMode;
    +		if (syncOrAsync == "async") {
    +			ioMode = IOManager.IOMode.ASYNC;
    +		} else {
    +			ioMode = IOManager.IOMode.SYNC;
    +		}
    +
    +		NetworkEnvironmentConfiguration networkConfig = new NetworkEnvironmentConfiguration(
    +			numNetworkBuffers,
    +			pageSize,
    +			memType,
    +			ioMode,
    +			nettyConfig == null ? Option.<NettyConfig>empty() : new Some<>(nettyConfig),
    +			new Tuple2<>(500, 3000));
    +
    +		// ----> timeouts, library caching, profiling
    +
    +		FiniteDuration timeout;
    +		try {
    +			timeout = AkkaUtils.getTimeout(configuration);
    +		} catch (Exception e) {
    +			throw new IllegalArgumentException(
    +				"Invalid format for '" + ConfigConstants.AKKA_ASK_TIMEOUT +
    +					"'.Use formats like '50 s' or '1 min' to specify the timeout.");
    +		}
    +		LOG.info("Messages between TaskManager and JobManager have a max timeout of " + timeout);
    +
    +		long cleanupInterval = configuration.getLong(
    +			ConfigConstants.LIBRARY_CACHE_MANAGER_CLEANUP_INTERVAL,
    +			ConfigConstants.DEFAULT_LIBRARY_CACHE_MANAGER_CLEANUP_INTERVAL) * 1000;
    +
    +		FiniteDuration finiteRegistrationDuration = null;
    +		try {
    +			Duration maxRegistrationDuration = Duration.create(configuration.getString(
    +				ConfigConstants.TASK_MANAGER_MAX_REGISTRATION_DURATION,
    +				ConfigConstants.DEFAULT_TASK_MANAGER_MAX_REGISTRATION_DURATION));
    +			if (maxRegistrationDuration.isFinite()) {
    +				finiteRegistrationDuration = new FiniteDuration(maxRegistrationDuration.toSeconds(), TimeUnit.SECONDS);
    +			}
    +		} catch (NumberFormatException e) {
    +			throw new IllegalArgumentException("Invalid format for parameter " +
    +				ConfigConstants.TASK_MANAGER_MAX_REGISTRATION_DURATION, e);
    +		}
    +
    +		FiniteDuration initialRegistrationPause;
    --- End diff --
    
    `final`


---
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.
---

[GitHub] flink pull request #2400: [FLINK-4363] Implement TaskManager basic startup o...

Posted by mxm <gi...@git.apache.org>.
Github user mxm commented on a diff in the pull request:

    https://github.com/apache/flink/pull/2400#discussion_r76085144
  
    --- Diff: flink-runtime/src/main/java/org/apache/flink/runtime/rpc/taskexecutor/TaskExecutor.java ---
    @@ -94,12 +729,11 @@ public void notifyOfNewResourceManagerLeader(String newLeaderAddress, UUID newLe
     			if (newLeaderAddress != null) {
     				// the resource manager switched to a new leader
     				log.info("ResourceManager leader changed from {} to {}. Registering at new leader.",
    -						resourceManagerConnection.getResourceManagerAddress(), newLeaderAddress);
    --- End diff --
    
    Same goes for the start method.


---
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.
---

[GitHub] flink pull request #2400: [FLINK-4363] Implement TaskManager basic startup o...

Posted by mxm <gi...@git.apache.org>.
Github user mxm commented on a diff in the pull request:

    https://github.com/apache/flink/pull/2400#discussion_r75646889
  
    --- Diff: flink-runtime/src/main/java/org/apache/flink/runtime/rpc/taskexecutor/TaskExecutor.java ---
    @@ -35,27 +79,634 @@
      */
     public class TaskExecutor extends RpcEndpoint<TaskExecutorGateway> {
     
    -	/** The unique resource ID of this TaskExecutor */
    +	private static final Logger LOG = LoggerFactory.getLogger(TaskExecutor.class);
    +
    +	/** Return code for critical errors during the runtime */
    +	private static final int RUNTIME_FAILURE_RETURN_CODE = 2;
    +
    +	/** The name of the TaskManager actor */
    +	private static final String TASK_MANAGER_NAME = "taskmanager";
    +
    +	/** The unique resource ID of this TaskManager */
     	private final ResourceID resourceID;
     
     	/** The access to the leader election and metadata storage services */
     	private final HighAvailabilityServices haServices;
     
    -	// --------- resource manager --------
    +	/** The task manager configuration */
    +	private final TaskManagerConfiguration taskManagerConfig;
    +
    +	/** The connection information of the task manager */
    +	private final InstanceConnectionInfo connectionInfo;
    +
    +	/** The I/O manager component in the task manager */
    +	private final IOManager ioManager;
    +
    +	/** The memory manager component in the task manager */
    +	private final MemoryManager memoryManager;
    +
    +	/** The network component in the task manager */
    +	private final NetworkEnvironment networkEnvironment;
    +
    +	/** The number of slots in the task manager, should be 1 for YARN */
    +	private final int numberOfSlots;
     
    +	// --------- resource manager --------
     	private TaskExecutorToResourceManagerConnection resourceManagerConnection;
     
     	// ------------------------------------------------------------------------
     
     	public TaskExecutor(
    +			TaskManagerConfiguration taskManagerConfig,
    +			ResourceID resourceID,
    +			InstanceConnectionInfo connectionInfo,
    +			MemoryManager memoryManager,
    +			IOManager ioManager,
    +			NetworkEnvironment networkEnvironment,
    +			int numberOfSlots,
     			RpcService rpcService,
    -			HighAvailabilityServices haServices,
    -			ResourceID resourceID) {
    +			HighAvailabilityServices haServices) {
     
     		super(rpcService);
     
    -		this.haServices = checkNotNull(haServices);
    +		this.taskManagerConfig = checkNotNull(taskManagerConfig);
     		this.resourceID = checkNotNull(resourceID);
    +		this.connectionInfo = checkNotNull(connectionInfo);
    +		this.memoryManager = checkNotNull(memoryManager);
    +		this.ioManager = checkNotNull(ioManager);
    +		this.networkEnvironment = checkNotNull(networkEnvironment);
    +		this.numberOfSlots = checkNotNull(numberOfSlots);
    +		this.haServices = checkNotNull(haServices);
    +	}
    +
    +	/**
    +	 * Starts and runs the TaskManager.
    +	 * <p/>
    +	 * This method first tries to select the network interface to use for the TaskManager
    +	 * communication. The network interface is used both for the actor communication
    +	 * (coordination) as well as for the data exchange between task managers. Unless
    +	 * the hostname/interface is explicitly configured in the configuration, this
    +	 * method will try out various interfaces and methods to connect to the JobManager
    +	 * and select the one where the connection attempt is successful.
    +	 * <p/>
    +	 * After selecting the network interface, this method brings up an actor system
    +	 * for the TaskManager and its actors, starts the TaskManager's services
    +	 * (library cache, shuffle network stack, ...), and starts the TaskManager itself.
    +	 *
    +	 * @param configuration    The configuration for the TaskManager.
    +	 * @param taskManagerClass The actor class to instantiate.
    +	 *                         Allows to use TaskManager subclasses for example for YARN.
    +	 */
    +	public static void selectNetworkInterfaceAndRunTaskManager(
    +		Configuration configuration,
    +		ResourceID resourceID,
    +		Class<? extends TaskManager> taskManagerClass) throws Exception {
    +
    +		Tuple2<String, Integer> tuple2 = selectNetworkInterfaceAndPort(configuration);
    +
    +		runTaskManager(tuple2._1(), resourceID, tuple2._2(), configuration, taskManagerClass);
    +	}
    +
    +	private static Tuple2<String, Integer> selectNetworkInterfaceAndPort(Configuration configuration)
    +		throws Exception {
    +		String taskManagerHostname = configuration.getString(ConfigConstants.TASK_MANAGER_HOSTNAME_KEY, null);
    +		if (taskManagerHostname != null) {
    +			LOG.info("Using configured hostname/address for TaskManager: " + taskManagerHostname);
    +		} else {
    +			LeaderRetrievalService leaderRetrievalService = LeaderRetrievalUtils.createLeaderRetrievalService(configuration);
    +			FiniteDuration lookupTimeout = AkkaUtils.getLookupTimeout(configuration);
    +
    +			InetAddress taskManagerAddress = LeaderRetrievalUtils.findConnectingAddress(leaderRetrievalService, lookupTimeout);
    +			taskManagerHostname = taskManagerAddress.getHostName();
    +			LOG.info("TaskManager will use hostname/address '{}' ({}) for communication.",
    +				taskManagerHostname, taskManagerAddress.getHostAddress());
    +		}
    +
    +		// if no task manager port has been configured, use 0 (system will pick any free port)
    +		int actorSystemPort = configuration.getInteger(ConfigConstants.TASK_MANAGER_IPC_PORT_KEY, 0);
    +		if (actorSystemPort < 0 || actorSystemPort > 65535) {
    +			throw new IllegalConfigurationException("Invalid value for '" +
    +				ConfigConstants.TASK_MANAGER_IPC_PORT_KEY +
    +				"' (port for the TaskManager actor system) : " + actorSystemPort +
    +				" - Leave config parameter empty or use 0 to let the system choose a port automatically.");
    +		}
    +
    +		return new Tuple2<>(taskManagerHostname, actorSystemPort);
    +	}
    +
    +	/**
    +	 * Starts and runs the TaskManager. Brings up an actor system for the TaskManager and its
    +	 * actors, starts the TaskManager's services (library cache, shuffle network stack, ...),
    +	 * and starts the TaskManager itself.
    +	 * <p/>
    +	 * This method will also spawn a process reaper for the TaskManager (kill the process if
    +	 * the actor fails) and optionally start the JVM memory logging thread.
    +	 *
    +	 * @param taskManagerHostname The hostname/address of the interface where the actor system
    +	 *                            will communicate.
    +	 * @param resourceID          The id of the resource which the task manager will run on.
    +	 * @param actorSystemPort     The port at which the actor system will communicate.
    +	 * @param configuration       The configuration for the TaskManager.
    +	 * @param taskManagerClass    The actor class to instantiate. Allows the use of TaskManager
    +	 *                            subclasses for example for YARN.
    +	 */
    +	private static void runTaskManager(
    +		String taskManagerHostname,
    +		ResourceID resourceID,
    +		int actorSystemPort,
    +		Configuration configuration,
    +		Class<? extends TaskManager> taskManagerClass) throws Exception {
    +
    +		LOG.info("Starting TaskManager");
    +
    +		// Bring up the TaskManager actor system first, bind it to the given address.
    +
    +		LOG.info("Starting TaskManager actor system at " +
    +			NetUtils.hostAndPortToUrlString(taskManagerHostname, actorSystemPort));
    +
    +		ActorSystem taskManagerSystem;
    --- End diff --
    
    Would make this final.


---
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.
---

[GitHub] flink pull request #2400: [FLINK-4363] Implement TaskManager basic startup o...

Posted by wangzhijiang999 <gi...@git.apache.org>.
Github user wangzhijiang999 commented on a diff in the pull request:

    https://github.com/apache/flink/pull/2400#discussion_r76213620
  
    --- Diff: flink-runtime/src/main/java/org/apache/flink/runtime/rpc/taskexecutor/TaskExecutor.java ---
    @@ -36,27 +82,617 @@
      */
     public class TaskExecutor extends RpcEndpoint<TaskExecutorGateway> {
     
    -	/** The unique resource ID of this TaskExecutor */
    +	private static final Logger LOG = LoggerFactory.getLogger(TaskExecutor.class);
    +
    +	/** The unique resource ID of this TaskManager */
     	private final ResourceID resourceID;
     
     	/** The access to the leader election and metadata storage services */
     	private final HighAvailabilityServices haServices;
     
    -	// --------- resource manager --------
    +	/** The task manager configuration */
    +	private final TaskExecutorConfiguration taskExecutorConfig;
    +
    +	/** The connection information of the task manager */
    +	private final InstanceConnectionInfo connectionInfo;
    +
    +	/** The I/O manager component in the task manager */
    +	private final IOManager ioManager;
    +
    +	/** The memory manager component in the task manager */
    +	private final MemoryManager memoryManager;
    +
    +	/** The network component in the task manager */
    +	private final NetworkEnvironment networkEnvironment;
    +
    +	/** The number of slots in the task manager, should be 1 for YARN */
    +	private final int numberOfSlots;
     
    +	// --------- resource manager --------
     	private TaskExecutorToResourceManagerConnection resourceManagerConnection;
     
     	// ------------------------------------------------------------------------
     
     	public TaskExecutor(
    +			TaskExecutorConfiguration taskExecutorConfig,
    +			ResourceID resourceID,
    +			InstanceConnectionInfo connectionInfo,
    +			MemoryManager memoryManager,
    +			IOManager ioManager,
    +			NetworkEnvironment networkEnvironment,
    +			int numberOfSlots,
     			RpcService rpcService,
    -			HighAvailabilityServices haServices,
    -			ResourceID resourceID) {
    +			HighAvailabilityServices haServices) {
     
     		super(rpcService);
     
    -		this.haServices = checkNotNull(haServices);
    +		this.taskExecutorConfig = checkNotNull(taskExecutorConfig);
     		this.resourceID = checkNotNull(resourceID);
    +		this.connectionInfo = checkNotNull(connectionInfo);
    +		this.memoryManager = checkNotNull(memoryManager);
    +		this.ioManager = checkNotNull(ioManager);
    +		this.networkEnvironment = checkNotNull(networkEnvironment);
    +		this.numberOfSlots = checkNotNull(numberOfSlots);
    +		this.haServices = checkNotNull(haServices);
    +	}
    +
    +	/**
    +	 * Starts and runs the TaskManager.
    +	 * <p/>
    +	 * This method first tries to select the network interface to use for the TaskManager
    +	 * communication. The network interface is used both for the actor communication
    +	 * (coordination) as well as for the data exchange between task managers. Unless
    +	 * the hostname/interface is explicitly configured in the configuration, this
    +	 * method will try out various interfaces and methods to connect to the JobManager
    +	 * and select the one where the connection attempt is successful.
    +	 * <p/>
    +	 * After selecting the network interface, this method brings up an actor system
    +	 * for the TaskManager and its actors, starts the TaskManager's services
    +	 * (library cache, shuffle network stack, ...), and starts the TaskManager itself.
    +	 *
    +	 * @param configuration    The configuration for the TaskManager.
    +	 * @param resourceID       The id of the resource which the task manager will run on.
    +	 */
    +	public static void selectNetworkInterfaceAndRunTaskManager(
    +		Configuration configuration,
    +		ResourceID resourceID) throws Exception {
    +
    +		InetSocketAddress taskManagerAddress = selectNetworkInterfaceAndPort(configuration);
    +
    +		runTaskManager(taskManagerAddress.getHostName(), resourceID, taskManagerAddress.getPort(), configuration);
    +	}
    +
    +	private static InetSocketAddress selectNetworkInterfaceAndPort(Configuration configuration)
    +		throws Exception {
    +		String taskManagerHostname = configuration.getString(ConfigConstants.TASK_MANAGER_HOSTNAME_KEY, null);
    +		if (taskManagerHostname != null) {
    +			LOG.info("Using configured hostname/address for TaskManager: " + taskManagerHostname);
    +		} else {
    +			LeaderRetrievalService leaderRetrievalService = LeaderRetrievalUtils.createLeaderRetrievalService(configuration);
    +			FiniteDuration lookupTimeout = AkkaUtils.getLookupTimeout(configuration);
    +
    +			InetAddress taskManagerAddress = LeaderRetrievalUtils.findConnectingAddress(leaderRetrievalService, lookupTimeout);
    +			taskManagerHostname = taskManagerAddress.getHostName();
    +			LOG.info("TaskManager will use hostname/address '{}' ({}) for communication.",
    +				taskManagerHostname, taskManagerAddress.getHostAddress());
    +		}
    +
    +		// if no task manager port has been configured, use 0 (system will pick any free port)
    +		int actorSystemPort = configuration.getInteger(ConfigConstants.TASK_MANAGER_IPC_PORT_KEY, 0);
    +		if (actorSystemPort < 0 || actorSystemPort > 65535) {
    +			throw new IllegalConfigurationException("Invalid value for '" +
    +				ConfigConstants.TASK_MANAGER_IPC_PORT_KEY +
    +				"' (port for the TaskManager actor system) : " + actorSystemPort +
    +				" - Leave config parameter empty or use 0 to let the system choose a port automatically.");
    +		}
    +
    +		return new InetSocketAddress(taskManagerHostname, actorSystemPort);
    +	}
    +
    +	/**
    +	 * Starts and runs the TaskManager. Brings up an actor system for the TaskManager and its
    +	 * actors, starts the TaskManager's services (library cache, shuffle network stack, ...),
    +	 * and starts the TaskManager itself.
    +	 * <p/>
    +	 * This method will also spawn a process reaper for the TaskManager (kill the process if
    +	 * the actor fails) and optionally start the JVM memory logging thread.
    +	 *
    +	 * @param taskManagerHostname The hostname/address of the interface where the actor system
    +	 *                            will communicate.
    +	 * @param resourceID          The id of the resource which the task manager will run on.
    +	 * @param actorSystemPort   The port at which the actor system will communicate.
    +	 * @param configuration       The configuration for the TaskManager.
    +	 */
    +	private static void runTaskManager(
    +		String taskManagerHostname,
    +		ResourceID resourceID,
    +		int actorSystemPort,
    +		final Configuration configuration) throws Exception {
    +
    +		LOG.info("Starting TaskManager");
    +
    +		// Bring up the TaskManager actor system first, bind it to the given address.
    +
    +		LOG.info("Starting TaskManager actor system at " +
    +			NetUtils.hostAndPortToUrlString(taskManagerHostname, actorSystemPort));
    +
    +		final ActorSystem taskManagerSystem;
    +		try {
    +			Tuple2<String, Object> address = new Tuple2<String, Object>(taskManagerHostname, actorSystemPort);
    +			Config akkaConfig = AkkaUtils.getAkkaConfig(configuration, new Some<>(address));
    +			LOG.debug("Using akka configuration\n " + akkaConfig);
    +			taskManagerSystem = AkkaUtils.createActorSystem(akkaConfig);
    +		} catch (Throwable t) {
    +			if (t instanceof org.jboss.netty.channel.ChannelException) {
    +				Throwable cause = t.getCause();
    +				if (cause != null && t.getCause() instanceof java.net.BindException) {
    +					String address = NetUtils.hostAndPortToUrlString(taskManagerHostname, actorSystemPort);
    +					throw new IOException("Unable to bind TaskManager actor system to address " +
    +						address + " - " + cause.getMessage(), t);
    +				}
    +			}
    +			throw new Exception("Could not create TaskManager actor system", t);
    +		}
    +
    +		// start akka rpc service based on actor system
    +		Timeout timeout = new Timeout(AkkaUtils.getTimeout(configuration).toMillis(), TimeUnit.MILLISECONDS);
    +		AkkaRpcService akkaRpcService = new AkkaRpcService(taskManagerSystem, timeout);
    +
    +		// start high availability service to implement getResourceManagerLeaderRetriever method only
    +		HighAvailabilityServices haServices = new HighAvailabilityServices() {
    +			@Override
    +			public LeaderRetrievalService getResourceManagerLeaderRetriever() throws Exception {
    +				return LeaderRetrievalUtils.createLeaderRetrievalService(configuration);
    +			}
    +
    +			@Override
    +			public LeaderElectionService getJobMasterLeaderElectionService(JobID jobID) throws Exception {
    +				return null;
    +			}
    +		};
    +
    +		// start all the TaskManager services (network stack,  library cache, ...)
    +		// and the TaskManager actor
    +		try {
    +			LOG.info("Starting TaskManager actor");
    +			TaskExecutor taskExecutor = startTaskManagerComponentsAndActor(
    +				configuration,
    +				resourceID,
    +				akkaRpcService,
    +				taskManagerHostname,
    +				haServices,
    +				false);
    +
    +			taskExecutor.start();
    +
    +			// if desired, start the logging daemon that periodically logs the memory usage information
    +			if (LOG.isInfoEnabled() && configuration.getBoolean(
    +				ConfigConstants.TASK_MANAGER_DEBUG_MEMORY_USAGE_START_LOG_THREAD,
    +				ConfigConstants.DEFAULT_TASK_MANAGER_DEBUG_MEMORY_USAGE_START_LOG_THREAD)) {
    +				LOG.info("Starting periodic memory usage logger");
    +
    +				long interval = configuration.getLong(
    +					ConfigConstants.TASK_MANAGER_DEBUG_MEMORY_USAGE_LOG_INTERVAL_MS,
    +					ConfigConstants.DEFAULT_TASK_MANAGER_DEBUG_MEMORY_USAGE_LOG_INTERVAL_MS);
    +
    +				MemoryLogger logger = new MemoryLogger(LOG, interval, taskManagerSystem);
    +				logger.start();
    +			}
    +
    +			// block until everything is done
    +			taskManagerSystem.awaitTermination();
    +		} catch (Throwable t) {
    +			LOG.error("Error while starting up taskManager", t);
    +			try {
    +				taskManagerSystem.shutdown();
    +			} catch (Throwable tt) {
    +				LOG.warn("Could not cleanly shut down actor system", tt);
    +			}
    +			throw t;
    +		}
    +	}
    +
    +	// --------------------------------------------------------------------------
    +	//  Starting and running the TaskManager
    +	// --------------------------------------------------------------------------
    +
    +	/**
    +	 * @param configuration                 The configuration for the TaskManager.
    +	 * @param resourceID                    The id of the resource which the task manager will run on.
    +	 * @param rpcService                  The rpc service which is used to start and connect to the TaskManager RpcEndpoint .
    +	 * @param taskManagerHostname       The hostname/address that describes the TaskManager's data location.
    +	 * @param haServices        Optionally, a high availability service can be provided. If none is given,
    +	 *                                      then a HighAvailabilityServices is constructed from the configuration.
    +	 * @param localTaskManagerCommunication If true, the TaskManager will not initiate the TCP network stack.
    +	 * @return An ActorRef to the TaskManager actor.
    +	 * @throws org.apache.flink.configuration.IllegalConfigurationException     Thrown, if the given config contains illegal values.
    +	 * @throws java.io.IOException      Thrown, if any of the I/O components (such as buffer pools,
    +	 *                                       I/O manager, ...) cannot be properly started.
    +	 * @throws java.lang.Exception      Thrown is some other error occurs while parsing the configuration
    +	 *                                      or starting the TaskManager components.
    +	 */
    +	public static TaskExecutor startTaskManagerComponentsAndActor(
    +		Configuration configuration,
    +		ResourceID resourceID,
    +		RpcService rpcService,
    +		String taskManagerHostname,
    +		HighAvailabilityServices haServices,
    +		boolean localTaskManagerCommunication) throws Exception {
    +
    +		Tuple4<TaskExecutorConfiguration, NetworkEnvironmentConfiguration, InstanceConnectionInfo, MemoryType> tuple4
    +			= parseTaskManagerConfiguration(configuration, taskManagerHostname, localTaskManagerCommunication);
    +
    +		TaskExecutorConfiguration taskExecutorConfig = tuple4.f0;
    +		NetworkEnvironmentConfiguration netConfig = tuple4.f1;
    +		InstanceConnectionInfo connectionInfo = tuple4.f2;
    +		MemoryType memType = tuple4.f3;
    +
    +		// pre-start checks
    +		checkTempDirs(taskExecutorConfig.getTmpDirPaths());
    +
    +		ExecutionContext executionContext = ExecutionContexts$.MODULE$.fromExecutor(new ForkJoinPool());
    +
    +		// we start the network first, to make sure it can allocate its buffers first
    +		NetworkEnvironment network = new NetworkEnvironment(executionContext, taskExecutorConfig.getTimeout(), netConfig);
    +
    +		// computing the amount of memory to use depends on how much memory is available
    +		// it strictly needs to happen AFTER the network stack has been initialized
    +
    +		// check if a value has been configured
    +		long configuredMemory = configuration.getLong(ConfigConstants.TASK_MANAGER_MEMORY_SIZE_KEY, -1L);
    +		checkConfigParameter(configuredMemory == -1 || configuredMemory > 0, configuredMemory,
    +			ConfigConstants.TASK_MANAGER_MEMORY_SIZE_KEY,
    +			"MemoryManager needs at least one MB of memory. " +
    +				"If you leave this config parameter empty, the system automatically " +
    +				"pick a fraction of the available memory.");
    +
    +		long memorySize;
    +		boolean preAllocateMemory = configuration.getBoolean(
    +			ConfigConstants.TASK_MANAGER_MEMORY_PRE_ALLOCATE_KEY,
    +			ConfigConstants.DEFAULT_TASK_MANAGER_MEMORY_PRE_ALLOCATE);
    +		if (configuredMemory > 0) {
    +			if (preAllocateMemory) {
    +				LOG.info("Using {} MB for managed memory." , configuredMemory);
    +			} else {
    +				LOG.info("Limiting managed memory to {} MB, memory will be allocated lazily." , configuredMemory);
    +			}
    +			memorySize = configuredMemory << 20; // megabytes to bytes
    +		} else {
    +			float fraction = configuration.getFloat(
    +				ConfigConstants.TASK_MANAGER_MEMORY_FRACTION_KEY,
    +				ConfigConstants.DEFAULT_MEMORY_MANAGER_MEMORY_FRACTION);
    +			checkConfigParameter(fraction > 0.0f && fraction < 1.0f, fraction,
    +				ConfigConstants.TASK_MANAGER_MEMORY_FRACTION_KEY,
    +				"MemoryManager fraction of the free memory must be between 0.0 and 1.0");
    +
    +			if (memType == MemoryType.HEAP) {
    +				long relativeMemSize = (long) (EnvironmentInformation.getSizeOfFreeHeapMemoryWithDefrag() * fraction);
    +				if (preAllocateMemory) {
    +					LOG.info("Using {} of the currently free heap space for managed heap memory ({} MB)." ,
    +						fraction , relativeMemSize >> 20);
    +				} else {
    +					LOG.info("Limiting managed memory to {} of the currently free heap space ({} MB), " +
    +						"memory will be allocated lazily." , fraction , relativeMemSize >> 20);
    +				}
    +				memorySize = relativeMemSize;
    +			} else if (memType == MemoryType.OFF_HEAP) {
    +				// The maximum heap memory has been adjusted according to the fraction
    +				long maxMemory = EnvironmentInformation.getMaxJvmHeapMemory();
    +				long directMemorySize = (long) (maxMemory / (1.0 - fraction) * fraction);
    +				if (preAllocateMemory) {
    +					LOG.info("Using {} of the maximum memory size for managed off-heap memory ({} MB)." ,
    +						fraction, directMemorySize >> 20);
    +				} else {
    +					LOG.info("Limiting managed memory to {} of the maximum memory size ({} MB)," +
    +						" memory will be allocated lazily.", fraction, directMemorySize >> 20);
    +				}
    +				memorySize = directMemorySize;
    +			} else {
    +				throw new RuntimeException("No supported memory type detected.");
    +			}
    +		}
    +
    +		// now start the memory manager
    +		final MemoryManager memoryManager;
    +		try {
    +			memoryManager = new MemoryManager(
    +				memorySize,
    +				taskExecutorConfig.getNumberOfSlots(),
    +				netConfig.networkBufferSize(),
    +				memType,
    +				preAllocateMemory);
    +		} catch (OutOfMemoryError e) {
    +			if (memType == MemoryType.HEAP) {
    +				throw new Exception("OutOfMemory error (" + e.getMessage() +
    +					") while allocating the TaskManager heap memory (" + memorySize + " bytes).", e);
    +			} else if (memType == MemoryType.OFF_HEAP) {
    +				throw new Exception("OutOfMemory error (" + e.getMessage() +
    +					") while allocating the TaskManager off-heap memory (" + memorySize +
    +					" bytes).Try increasing the maximum direct memory (-XX:MaxDirectMemorySize)", e);
    +			} else {
    +				throw e;
    +			}
    +		}
    +
    +		// start the I/O manager, it will create some temp directories.
    +		IOManager ioManager = new IOManagerAsync(taskExecutorConfig.getTmpDirPaths());
    +
    +		TaskExecutor taskExecutor = new TaskExecutor(
    +			taskExecutorConfig,
    +			resourceID,
    +			connectionInfo,
    +			memoryManager,
    +			ioManager,
    +			network,
    +			taskExecutorConfig.getNumberOfSlots(),
    +			rpcService,
    +			haServices);
    +
    +		return taskExecutor;
    +	}
    +
    +	// --------------------------------------------------------------------------
    +	//  Parsing and checking the TaskManager Configuration
    +	// --------------------------------------------------------------------------
    +
    +	/**
    +	 * Utility method to extract TaskManager config parameters from the configuration and to
    +	 * sanity check them.
    +	 *
    +	 * @param configuration                 The configuration.
    +	 * @param taskManagerHostname           The host name under which the TaskManager communicates.
    +	 * @param localTaskManagerCommunication True, to skip initializing the network stack.
    +	 *                                      Use only in cases where only one task manager runs.
    +	 * @return A tuple (TaskManagerConfiguration, network configuration,
    +	 * InstanceConnectionInfo, JobManager actor Akka URL).
    +	 */
    +	private static Tuple4<TaskExecutorConfiguration, NetworkEnvironmentConfiguration, InstanceConnectionInfo, MemoryType>
    --- End diff --
    
    Yes, I will push other modifications soon for your merge, then I can create another jira for the factory issue.


---
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.
---

[GitHub] flink pull request #2400: [FLINK-4363] Implement TaskManager basic startup o...

Posted by mxm <gi...@git.apache.org>.
Github user mxm commented on a diff in the pull request:

    https://github.com/apache/flink/pull/2400#discussion_r75648889
  
    --- Diff: flink-runtime/src/main/java/org/apache/flink/runtime/rpc/taskexecutor/TaskExecutor.java ---
    @@ -35,27 +79,634 @@
      */
     public class TaskExecutor extends RpcEndpoint<TaskExecutorGateway> {
     
    -	/** The unique resource ID of this TaskExecutor */
    +	private static final Logger LOG = LoggerFactory.getLogger(TaskExecutor.class);
    +
    +	/** Return code for critical errors during the runtime */
    +	private static final int RUNTIME_FAILURE_RETURN_CODE = 2;
    +
    +	/** The name of the TaskManager actor */
    +	private static final String TASK_MANAGER_NAME = "taskmanager";
    --- End diff --
    
    Also elsewhere in the code it still says TaskManager.


---
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.
---

[GitHub] flink issue #2400: [FLINK-4363] Implement TaskManager basic startup of all c...

Posted by mxm <gi...@git.apache.org>.
Github user mxm commented on the issue:

    https://github.com/apache/flink/pull/2400
  
    @wangzhijiang999 Could you port your changes to the new flip-6 branch? We can then go ahead and merge this PR.


---
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.
---

[GitHub] flink pull request #2400: [FLINK-4363] Implement TaskManager basic startup o...

Posted by mxm <gi...@git.apache.org>.
Github user mxm commented on a diff in the pull request:

    https://github.com/apache/flink/pull/2400#discussion_r75647841
  
    --- Diff: flink-runtime/src/main/java/org/apache/flink/runtime/rpc/taskexecutor/TaskExecutor.java ---
    @@ -35,27 +79,634 @@
      */
     public class TaskExecutor extends RpcEndpoint<TaskExecutorGateway> {
     
    -	/** The unique resource ID of this TaskExecutor */
    +	private static final Logger LOG = LoggerFactory.getLogger(TaskExecutor.class);
    +
    +	/** Return code for critical errors during the runtime */
    +	private static final int RUNTIME_FAILURE_RETURN_CODE = 2;
    +
    +	/** The name of the TaskManager actor */
    +	private static final String TASK_MANAGER_NAME = "taskmanager";
    +
    +	/** The unique resource ID of this TaskManager */
     	private final ResourceID resourceID;
     
     	/** The access to the leader election and metadata storage services */
     	private final HighAvailabilityServices haServices;
     
    -	// --------- resource manager --------
    +	/** The task manager configuration */
    +	private final TaskManagerConfiguration taskManagerConfig;
    +
    +	/** The connection information of the task manager */
    +	private final InstanceConnectionInfo connectionInfo;
    +
    +	/** The I/O manager component in the task manager */
    +	private final IOManager ioManager;
    +
    +	/** The memory manager component in the task manager */
    +	private final MemoryManager memoryManager;
    +
    +	/** The network component in the task manager */
    +	private final NetworkEnvironment networkEnvironment;
    +
    +	/** The number of slots in the task manager, should be 1 for YARN */
    +	private final int numberOfSlots;
     
    +	// --------- resource manager --------
     	private TaskExecutorToResourceManagerConnection resourceManagerConnection;
     
     	// ------------------------------------------------------------------------
     
     	public TaskExecutor(
    +			TaskManagerConfiguration taskManagerConfig,
    +			ResourceID resourceID,
    +			InstanceConnectionInfo connectionInfo,
    +			MemoryManager memoryManager,
    +			IOManager ioManager,
    +			NetworkEnvironment networkEnvironment,
    +			int numberOfSlots,
     			RpcService rpcService,
    -			HighAvailabilityServices haServices,
    -			ResourceID resourceID) {
    +			HighAvailabilityServices haServices) {
     
     		super(rpcService);
     
    -		this.haServices = checkNotNull(haServices);
    +		this.taskManagerConfig = checkNotNull(taskManagerConfig);
     		this.resourceID = checkNotNull(resourceID);
    +		this.connectionInfo = checkNotNull(connectionInfo);
    +		this.memoryManager = checkNotNull(memoryManager);
    +		this.ioManager = checkNotNull(ioManager);
    +		this.networkEnvironment = checkNotNull(networkEnvironment);
    +		this.numberOfSlots = checkNotNull(numberOfSlots);
    +		this.haServices = checkNotNull(haServices);
    +	}
    +
    +	/**
    +	 * Starts and runs the TaskManager.
    +	 * <p/>
    +	 * This method first tries to select the network interface to use for the TaskManager
    +	 * communication. The network interface is used both for the actor communication
    +	 * (coordination) as well as for the data exchange between task managers. Unless
    +	 * the hostname/interface is explicitly configured in the configuration, this
    +	 * method will try out various interfaces and methods to connect to the JobManager
    +	 * and select the one where the connection attempt is successful.
    +	 * <p/>
    +	 * After selecting the network interface, this method brings up an actor system
    +	 * for the TaskManager and its actors, starts the TaskManager's services
    +	 * (library cache, shuffle network stack, ...), and starts the TaskManager itself.
    +	 *
    +	 * @param configuration    The configuration for the TaskManager.
    +	 * @param taskManagerClass The actor class to instantiate.
    +	 *                         Allows to use TaskManager subclasses for example for YARN.
    +	 */
    +	public static void selectNetworkInterfaceAndRunTaskManager(
    +		Configuration configuration,
    +		ResourceID resourceID,
    +		Class<? extends TaskManager> taskManagerClass) throws Exception {
    +
    +		Tuple2<String, Integer> tuple2 = selectNetworkInterfaceAndPort(configuration);
    +
    +		runTaskManager(tuple2._1(), resourceID, tuple2._2(), configuration, taskManagerClass);
    +	}
    +
    +	private static Tuple2<String, Integer> selectNetworkInterfaceAndPort(Configuration configuration)
    +		throws Exception {
    +		String taskManagerHostname = configuration.getString(ConfigConstants.TASK_MANAGER_HOSTNAME_KEY, null);
    +		if (taskManagerHostname != null) {
    +			LOG.info("Using configured hostname/address for TaskManager: " + taskManagerHostname);
    +		} else {
    +			LeaderRetrievalService leaderRetrievalService = LeaderRetrievalUtils.createLeaderRetrievalService(configuration);
    +			FiniteDuration lookupTimeout = AkkaUtils.getLookupTimeout(configuration);
    +
    +			InetAddress taskManagerAddress = LeaderRetrievalUtils.findConnectingAddress(leaderRetrievalService, lookupTimeout);
    +			taskManagerHostname = taskManagerAddress.getHostName();
    +			LOG.info("TaskManager will use hostname/address '{}' ({}) for communication.",
    +				taskManagerHostname, taskManagerAddress.getHostAddress());
    +		}
    +
    +		// if no task manager port has been configured, use 0 (system will pick any free port)
    +		int actorSystemPort = configuration.getInteger(ConfigConstants.TASK_MANAGER_IPC_PORT_KEY, 0);
    +		if (actorSystemPort < 0 || actorSystemPort > 65535) {
    +			throw new IllegalConfigurationException("Invalid value for '" +
    +				ConfigConstants.TASK_MANAGER_IPC_PORT_KEY +
    +				"' (port for the TaskManager actor system) : " + actorSystemPort +
    +				" - Leave config parameter empty or use 0 to let the system choose a port automatically.");
    +		}
    +
    +		return new Tuple2<>(taskManagerHostname, actorSystemPort);
    +	}
    +
    +	/**
    +	 * Starts and runs the TaskManager. Brings up an actor system for the TaskManager and its
    +	 * actors, starts the TaskManager's services (library cache, shuffle network stack, ...),
    +	 * and starts the TaskManager itself.
    +	 * <p/>
    +	 * This method will also spawn a process reaper for the TaskManager (kill the process if
    +	 * the actor fails) and optionally start the JVM memory logging thread.
    +	 *
    +	 * @param taskManagerHostname The hostname/address of the interface where the actor system
    +	 *                            will communicate.
    +	 * @param resourceID          The id of the resource which the task manager will run on.
    +	 * @param actorSystemPort     The port at which the actor system will communicate.
    +	 * @param configuration       The configuration for the TaskManager.
    +	 * @param taskManagerClass    The actor class to instantiate. Allows the use of TaskManager
    +	 *                            subclasses for example for YARN.
    +	 */
    +	private static void runTaskManager(
    +		String taskManagerHostname,
    +		ResourceID resourceID,
    +		int actorSystemPort,
    +		Configuration configuration,
    +		Class<? extends TaskManager> taskManagerClass) throws Exception {
    +
    +		LOG.info("Starting TaskManager");
    +
    +		// Bring up the TaskManager actor system first, bind it to the given address.
    +
    +		LOG.info("Starting TaskManager actor system at " +
    +			NetUtils.hostAndPortToUrlString(taskManagerHostname, actorSystemPort));
    +
    +		ActorSystem taskManagerSystem;
    +		try {
    +			Tuple2<String, Object> address = new Tuple2<String, Object>(taskManagerHostname, actorSystemPort);
    +			Config akkaConfig = AkkaUtils.getAkkaConfig(configuration, new Some<>(address));
    +			LOG.debug("Using akka configuration\n " + akkaConfig);
    +			taskManagerSystem = AkkaUtils.createActorSystem(akkaConfig);
    +		} catch (Throwable t) {
    +			if (t instanceof org.jboss.netty.channel.ChannelException) {
    +				Throwable cause = t.getCause();
    +				if (cause != null && t.getCause() instanceof java.net.BindException) {
    +					String address = NetUtils.hostAndPortToUrlString(taskManagerHostname, actorSystemPort);
    +					throw new IOException("Unable to bind TaskManager actor system to address " +
    +						address + " - " + cause.getMessage(), t);
    +				}
    +			}
    +			throw new Exception("Could not create TaskManager actor system", t);
    +		}
    +
    +		// start all the TaskManager services (network stack,  library cache, ...)
    +		// and the TaskManager actor
    +		try {
    +			LOG.info("Starting TaskManager actor");
    +			ActorRef taskManagerActor = startTaskManagerComponentsAndActor(
    +				configuration,
    +				resourceID,
    +				taskManagerSystem,
    +				taskManagerHostname,
    +				TASK_MANAGER_NAME,
    +				null,
    +				false,
    +				taskManagerClass);
    +
    +			// start a process reaper that watches the JobManager. If the TaskManager actor dies,
    +			// the process reaper will kill the JVM process (to ensure easy failure detection)
    +			LOG.debug("Starting TaskManager process reaper");
    +
    +			taskManagerSystem.actorOf(
    +				Props.create(ProcessReaper.class, taskManagerActor, LOG, RUNTIME_FAILURE_RETURN_CODE),
    +				"TaskManager_Process_Reaper");
    +
    +			// if desired, start the logging daemon that periodically logs the memory usage information
    +			if (LOG.isInfoEnabled() && configuration.getBoolean(
    +				ConfigConstants.TASK_MANAGER_DEBUG_MEMORY_USAGE_START_LOG_THREAD,
    +				ConfigConstants.DEFAULT_TASK_MANAGER_DEBUG_MEMORY_USAGE_START_LOG_THREAD)) {
    +				LOG.info("Starting periodic memory usage logger");
    +
    +				long interval = configuration.getLong(
    +					ConfigConstants.TASK_MANAGER_DEBUG_MEMORY_USAGE_LOG_INTERVAL_MS,
    +					ConfigConstants.DEFAULT_TASK_MANAGER_DEBUG_MEMORY_USAGE_LOG_INTERVAL_MS);
    +
    +				MemoryLogger logger = new MemoryLogger(LOG, interval, taskManagerSystem);
    +				logger.start();
    +			}
    +
    +			// block until everything is done
    +			taskManagerSystem.awaitTermination();
    +		} catch (Throwable t) {
    +			LOG.error("Error while starting up taskManager", t);
    +			try {
    +				taskManagerSystem.shutdown();
    +			} catch (Throwable tt) {
    +				LOG.warn("Could not cleanly shut down actor system", tt);
    +			}
    +			throw t;
    +		}
    +	}
    +
    +	// --------------------------------------------------------------------------
    +	//  Starting and running the TaskManager
    +	// --------------------------------------------------------------------------
    +
    +	/**
    +	 * @param configuration                 The configuration for the TaskManager.
    +	 * @param resourceID                    The id of the resource which the task manager will run on.
    +	 * @param actorSystem                  The actor system that should run the TaskManager actor.
    +	 * @param taskManagerHostname       The hostname/address that describes the TaskManager's data location.
    +	 * @param taskManagerActorName      Optionally the name of the TaskManager actor. If none is given,
    +	 *                                      the actor will use a random name.
    +	 * @param leaderRetrievalService        Optionally, a leader retrieval service can be provided. If none is given,
    +	 *                                      then a LeaderRetrievalService is constructed from the configuration.
    +	 * @param localTaskManagerCommunication If true, the TaskManager will not initiate the TCP network stack.
    +	 * @param taskManagerClass      The class of the TaskManager actor. May be used to give
    +	 *                                      subclasses that understand additional actor messages.
    +	 * @return An ActorRef to the TaskManager actor.
    +	 * @throws org.apache.flink.configuration.IllegalConfigurationException     Thrown, if the given config contains illegal values.
    +	 * @throws java.io.IOException      Thrown, if any of the I/O components (such as buffer pools,
    +	 *                                       I/O manager, ...) cannot be properly started.
    +	 * @throws java.lang.Exception      Thrown is some other error occurs while parsing the configuration
    +	 *                                      or starting the TaskManager components.
    +	 */
    +	public static ActorRef startTaskManagerComponentsAndActor(
    +		Configuration configuration,
    +		ResourceID resourceID,
    +		ActorSystem actorSystem,
    +		String taskManagerHostname,
    +		String taskManagerActorName,
    +		LeaderRetrievalService leaderRetrievalService,
    +		boolean localTaskManagerCommunication,
    +		Class<? extends TaskManager> taskManagerClass) throws Exception {
    +
    +		Tuple4<TaskManagerConfiguration, NetworkEnvironmentConfiguration, InstanceConnectionInfo, MemoryType> tuple4
    +			= parseTaskManagerConfiguration(configuration, taskManagerHostname, localTaskManagerCommunication);
    +
    +		TaskManagerConfiguration taskManagerConfig = tuple4._1();
    +		NetworkEnvironmentConfiguration netConfig = tuple4._2();
    +		InstanceConnectionInfo connectionInfo = tuple4._3();
    +		MemoryType memType = tuple4._4();
    +
    +		// pre-start checks
    +		checkTempDirs(taskManagerConfig.tmpDirPaths());
    +
    +		ExecutionContext executionContext = ExecutionContexts$.MODULE$.fromExecutor(new ForkJoinPool());
    +
    +		// we start the network first, to make sure it can allocate its buffers first
    +		NetworkEnvironment network = new NetworkEnvironment(executionContext, taskManagerConfig.timeout(), netConfig);
    +
    +		// computing the amount of memory to use depends on how much memory is available
    +		// it strictly needs to happen AFTER the network stack has been initialized
    +
    +		// check if a value has been configured
    +		long configuredMemory = configuration.getLong(ConfigConstants.TASK_MANAGER_MEMORY_SIZE_KEY, -1L);
    +		checkConfigParameter(configuredMemory == -1 || configuredMemory > 0, configuredMemory,
    +			ConfigConstants.TASK_MANAGER_MEMORY_SIZE_KEY,
    +			"MemoryManager needs at least one MB of memory. " +
    +				"If you leave this config parameter empty, the system automatically " +
    +				"pick a fraction of the available memory.");
    +
    +		long memorySize;
    +		boolean preAllocateMemory = configuration.getBoolean(
    +			ConfigConstants.TASK_MANAGER_MEMORY_PRE_ALLOCATE_KEY,
    +			ConfigConstants.DEFAULT_TASK_MANAGER_MEMORY_PRE_ALLOCATE);
    +		if (configuredMemory > 0) {
    +			if (preAllocateMemory) {
    +				LOG.info("Using {} MB for managed memory." , configuredMemory);
    +			} else {
    +				LOG.info("Limiting managed memory to {} MB, memory will be allocated lazily." , configuredMemory);
    +			}
    +			memorySize = configuredMemory << 20; // megabytes to bytes
    +		} else {
    +			float fraction = configuration.getFloat(
    +				ConfigConstants.TASK_MANAGER_MEMORY_FRACTION_KEY,
    +				ConfigConstants.DEFAULT_MEMORY_MANAGER_MEMORY_FRACTION);
    +			checkConfigParameter(fraction > 0.0f && fraction < 1.0f, fraction,
    +				ConfigConstants.TASK_MANAGER_MEMORY_FRACTION_KEY,
    +				"MemoryManager fraction of the free memory must be between 0.0 and 1.0");
    +
    +			if (memType == MemoryType.HEAP) {
    +				long relativeMemSize = (long) (EnvironmentInformation.getSizeOfFreeHeapMemoryWithDefrag() * fraction);
    +				if (preAllocateMemory) {
    +					LOG.info("Using {} of the currently free heap space for managed heap memory ({} MB)." ,
    +						fraction , relativeMemSize >> 20);
    +				} else {
    +					LOG.info("Limiting managed memory to {} of the currently free heap space ({} MB), " +
    +						"memory will be allocated lazily." , fraction , relativeMemSize >> 20);
    +				}
    +				memorySize = relativeMemSize;
    +			} else if (memType == MemoryType.OFF_HEAP) {
    +				// The maximum heap memory has been adjusted according to the fraction
    +				long maxMemory = EnvironmentInformation.getMaxJvmHeapMemory();
    +				long directMemorySize = (long) (maxMemory / (1.0 - fraction) * fraction);
    +				if (preAllocateMemory) {
    +					LOG.info("Using {} of the maximum memory size for managed off-heap memory ({} MB)." ,
    +						fraction, directMemorySize >> 20);
    +				} else {
    +					LOG.info("Limiting managed memory to {} of the maximum memory size ({} MB)," +
    +						" memory will be allocated lazily.", fraction, directMemorySize >> 20);
    +				}
    +				memorySize = directMemorySize;
    +			} else {
    +				throw new RuntimeException("No supported memory type detected.");
    +			}
    +		}
    +
    +		// now start the memory manager
    +		MemoryManager memoryManager;
    +		try {
    +			memoryManager = new MemoryManager(
    +				memorySize,
    +				taskManagerConfig.numberOfSlots(),
    +				netConfig.networkBufferSize(),
    +				memType,
    +				preAllocateMemory);
    +		} catch (OutOfMemoryError e) {
    +			if (memType == MemoryType.HEAP) {
    +				throw new Exception("OutOfMemory error (" + e.getMessage() +
    +					") while allocating the TaskManager heap memory (" + memorySize + " bytes).", e);
    +			} else if (memType == MemoryType.OFF_HEAP) {
    +				throw new Exception("OutOfMemory error (" + e.getMessage() +
    +					") while allocating the TaskManager off-heap memory (" + memorySize +
    +					" bytes).Try increasing the maximum direct memory (-XX:MaxDirectMemorySize)", e);
    +			} else {
    +				throw e;
    +			}
    +		}
    +
    +		// start the I/O manager, it will create some temp directories.
    +		IOManager ioManager = new IOManagerAsync(taskManagerConfig.tmpDirPaths());
    +
    +		if (leaderRetrievalService == null){
    +			leaderRetrievalService = LeaderRetrievalUtils.createLeaderRetrievalService(configuration);
    +		}
    +
    +		// create the actor properties (which define the actor constructor parameters)
    +		Props tmProps = Props.create(
    +			taskManagerClass,
    +			taskManagerConfig,
    +			resourceID,
    +			connectionInfo,
    +			memoryManager,
    +			ioManager,
    +			network,
    +			taskManagerConfig.numberOfSlots(),
    +			leaderRetrievalService);
    +
    +		ActorRef taskManagerActorRef;
    +		if (taskManagerActorName != null && !taskManagerActorName.equals("")) {
    +			taskManagerActorRef = actorSystem.actorOf(tmProps, taskManagerActorName);
    +		} else {
    +			taskManagerActorRef = actorSystem.actorOf(tmProps);
    +		}
    +
    +		return taskManagerActorRef;
    +	}
    +
    +	// --------------------------------------------------------------------------
    +	//  Parsing and checking the TaskManager Configuration
    +	// --------------------------------------------------------------------------
    +
    +	/**
    +	 * Utility method to extract TaskManager config parameters from the configuration and to
    +	 * sanity check them.
    +	 *
    +	 * @param configuration                 The configuration.
    +	 * @param taskManagerHostname           The host name under which the TaskManager communicates.
    +	 * @param localTaskManagerCommunication True, to skip initializing the network stack.
    +	 *                                      Use only in cases where only one task manager runs.
    +	 * @return A tuple (TaskManagerConfiguration, network configuration,
    +	 * InstanceConnectionInfo, JobManager actor Akka URL).
    +	 */
    +	private static Tuple4<TaskManagerConfiguration, NetworkEnvironmentConfiguration, InstanceConnectionInfo, MemoryType>
    +		parseTaskManagerConfiguration(Configuration configuration, String taskManagerHostname, boolean localTaskManagerCommunication)
    +		throws Exception {
    +
    +		// ------- read values from the config and check them ---------
    +		//                      (a lot of them)
    +
    +		// ----> hosts / ports for communication and data exchange
    +
    +		int dataport = configuration.getInteger(ConfigConstants.TASK_MANAGER_DATA_PORT_KEY,
    +			ConfigConstants.DEFAULT_TASK_MANAGER_DATA_PORT);
    +		if (dataport == 0) {
    +			dataport = NetUtils.getAvailablePort();
    +		}
    +		checkConfigParameter(dataport > 0, dataport, ConfigConstants.TASK_MANAGER_DATA_PORT_KEY,
    +			"Leave config parameter empty or use 0 to let the system choose a port automatically.");
    +
    +		InetAddress taskManagerAddress = InetAddress.getByName(taskManagerHostname);
    +		InstanceConnectionInfo connectionInfo = new InstanceConnectionInfo(taskManagerAddress, dataport);
    +
    +		// ----> memory / network stack (shuffles/broadcasts), task slots, temp directories
    +
    +		// we need this because many configs have been written with a "-1" entry
    +		int slots = configuration.getInteger(ConfigConstants.TASK_MANAGER_NUM_TASK_SLOTS, 1);
    +		if (slots == -1) {
    +			slots = 1;
    +		}
    +		checkConfigParameter(slots >= 1, slots, ConfigConstants.TASK_MANAGER_NUM_TASK_SLOTS,
    +			"Number of task slots must be at least one.");
    +
    +		int numNetworkBuffers = configuration.getInteger(
    +			ConfigConstants.TASK_MANAGER_NETWORK_NUM_BUFFERS_KEY,
    +			ConfigConstants.DEFAULT_TASK_MANAGER_NETWORK_NUM_BUFFERS);
    +		checkConfigParameter(numNetworkBuffers > 0, numNetworkBuffers,
    +			ConfigConstants.TASK_MANAGER_NETWORK_NUM_BUFFERS_KEY, "");
    +
    +		int pageSize = configuration.getInteger(
    +			ConfigConstants.TASK_MANAGER_MEMORY_SEGMENT_SIZE_KEY,
    +			ConfigConstants.DEFAULT_TASK_MANAGER_MEMORY_SEGMENT_SIZE);
    +		// check page size of for minimum size
    +		checkConfigParameter(pageSize >= MemoryManager.MIN_PAGE_SIZE, pageSize,
    +			ConfigConstants.TASK_MANAGER_MEMORY_SEGMENT_SIZE_KEY,
    +			"Minimum memory segment size is " + MemoryManager.MIN_PAGE_SIZE);
    +		// check page size for power of two
    +		checkConfigParameter(MathUtils.isPowerOf2(pageSize), pageSize,
    +			ConfigConstants.TASK_MANAGER_MEMORY_SEGMENT_SIZE_KEY,
    +			"Memory segment size must be a power of 2.");
    +
    +		// check whether we use heap or off-heap memory
    +		MemoryType memType;
    +		if (configuration.getBoolean(ConfigConstants.TASK_MANAGER_MEMORY_OFF_HEAP_KEY, false)) {
    +			memType = MemoryType.OFF_HEAP;
    +		} else {
    +			memType = MemoryType.HEAP;
    +		}
    +
    +		// initialize the memory segment factory accordingly
    +		if (memType == MemoryType.HEAP) {
    +			if (!MemorySegmentFactory.isInitialized()) {
    +				MemorySegmentFactory.initializeFactory(HeapMemorySegment.FACTORY);
    +			} else if (MemorySegmentFactory.getFactory() != HeapMemorySegment.FACTORY) {
    +				throw new Exception("Memory type is set to heap memory, but memory segment " +
    +					"factory has been initialized for off-heap memory segments");
    +			}
    +		} else {
    +			if (!MemorySegmentFactory.isInitialized()) {
    +				MemorySegmentFactory.initializeFactory(HybridMemorySegment.FACTORY);
    +			} else if (MemorySegmentFactory.getFactory() != HybridMemorySegment.FACTORY) {
    +				throw new Exception("Memory type is set to off-heap memory, but memory segment " +
    +					"factory has been initialized for heap memory segments");
    +			}
    +		}
    +
    +		String[] tmpDirs = configuration.getString(
    +			ConfigConstants.TASK_MANAGER_TMP_DIR_KEY,
    +			ConfigConstants.DEFAULT_TASK_MANAGER_TMP_PATH).split(",|" + File.pathSeparator);
    +
    +		NettyConfig nettyConfig = null;
    +		if (!localTaskManagerCommunication) {
    +			nettyConfig = new NettyConfig(connectionInfo.address(), connectionInfo.dataPort(), pageSize, slots, configuration);
    +		}
    +
    +		// Default spill I/O mode for intermediate results
    +		String syncOrAsync = configuration.getString(
    +			ConfigConstants.TASK_MANAGER_NETWORK_DEFAULT_IO_MODE,
    +			ConfigConstants.DEFAULT_TASK_MANAGER_NETWORK_DEFAULT_IO_MODE);
    +
    +		IOMode ioMode;
    --- End diff --
    
    Would use `final` here.


---
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.
---

[GitHub] flink pull request #2400: [FLINK-4363] Implement TaskManager basic startup o...

Posted by mxm <gi...@git.apache.org>.
Github user mxm commented on a diff in the pull request:

    https://github.com/apache/flink/pull/2400#discussion_r75680765
  
    --- Diff: flink-runtime/src/main/java/org/apache/flink/runtime/rpc/taskexecutor/TaskExecutor.java ---
    @@ -35,27 +79,634 @@
      */
     public class TaskExecutor extends RpcEndpoint<TaskExecutorGateway> {
     
    -	/** The unique resource ID of this TaskExecutor */
    +	private static final Logger LOG = LoggerFactory.getLogger(TaskExecutor.class);
    +
    +	/** Return code for critical errors during the runtime */
    +	private static final int RUNTIME_FAILURE_RETURN_CODE = 2;
    +
    +	/** The name of the TaskManager actor */
    +	private static final String TASK_MANAGER_NAME = "taskmanager";
    +
    +	/** The unique resource ID of this TaskManager */
     	private final ResourceID resourceID;
     
     	/** The access to the leader election and metadata storage services */
     	private final HighAvailabilityServices haServices;
     
    -	// --------- resource manager --------
    +	/** The task manager configuration */
    +	private final TaskManagerConfiguration taskManagerConfig;
    +
    +	/** The connection information of the task manager */
    +	private final InstanceConnectionInfo connectionInfo;
    +
    +	/** The I/O manager component in the task manager */
    +	private final IOManager ioManager;
    +
    +	/** The memory manager component in the task manager */
    +	private final MemoryManager memoryManager;
    +
    +	/** The network component in the task manager */
    +	private final NetworkEnvironment networkEnvironment;
    +
    +	/** The number of slots in the task manager, should be 1 for YARN */
    +	private final int numberOfSlots;
     
    +	// --------- resource manager --------
     	private TaskExecutorToResourceManagerConnection resourceManagerConnection;
     
     	// ------------------------------------------------------------------------
     
     	public TaskExecutor(
    +			TaskManagerConfiguration taskManagerConfig,
    +			ResourceID resourceID,
    +			InstanceConnectionInfo connectionInfo,
    +			MemoryManager memoryManager,
    +			IOManager ioManager,
    +			NetworkEnvironment networkEnvironment,
    +			int numberOfSlots,
     			RpcService rpcService,
    -			HighAvailabilityServices haServices,
    -			ResourceID resourceID) {
    +			HighAvailabilityServices haServices) {
     
     		super(rpcService);
     
    -		this.haServices = checkNotNull(haServices);
    +		this.taskManagerConfig = checkNotNull(taskManagerConfig);
     		this.resourceID = checkNotNull(resourceID);
    +		this.connectionInfo = checkNotNull(connectionInfo);
    +		this.memoryManager = checkNotNull(memoryManager);
    +		this.ioManager = checkNotNull(ioManager);
    +		this.networkEnvironment = checkNotNull(networkEnvironment);
    +		this.numberOfSlots = checkNotNull(numberOfSlots);
    +		this.haServices = checkNotNull(haServices);
    +	}
    +
    +	/**
    +	 * Starts and runs the TaskManager.
    +	 * <p/>
    +	 * This method first tries to select the network interface to use for the TaskManager
    +	 * communication. The network interface is used both for the actor communication
    +	 * (coordination) as well as for the data exchange between task managers. Unless
    +	 * the hostname/interface is explicitly configured in the configuration, this
    +	 * method will try out various interfaces and methods to connect to the JobManager
    +	 * and select the one where the connection attempt is successful.
    +	 * <p/>
    +	 * After selecting the network interface, this method brings up an actor system
    +	 * for the TaskManager and its actors, starts the TaskManager's services
    +	 * (library cache, shuffle network stack, ...), and starts the TaskManager itself.
    +	 *
    +	 * @param configuration    The configuration for the TaskManager.
    +	 * @param taskManagerClass The actor class to instantiate.
    +	 *                         Allows to use TaskManager subclasses for example for YARN.
    +	 */
    +	public static void selectNetworkInterfaceAndRunTaskManager(
    +		Configuration configuration,
    +		ResourceID resourceID,
    +		Class<? extends TaskManager> taskManagerClass) throws Exception {
    +
    +		Tuple2<String, Integer> tuple2 = selectNetworkInterfaceAndPort(configuration);
    +
    +		runTaskManager(tuple2._1(), resourceID, tuple2._2(), configuration, taskManagerClass);
    +	}
    +
    +	private static Tuple2<String, Integer> selectNetworkInterfaceAndPort(Configuration configuration)
    +		throws Exception {
    +		String taskManagerHostname = configuration.getString(ConfigConstants.TASK_MANAGER_HOSTNAME_KEY, null);
    +		if (taskManagerHostname != null) {
    +			LOG.info("Using configured hostname/address for TaskManager: " + taskManagerHostname);
    +		} else {
    +			LeaderRetrievalService leaderRetrievalService = LeaderRetrievalUtils.createLeaderRetrievalService(configuration);
    +			FiniteDuration lookupTimeout = AkkaUtils.getLookupTimeout(configuration);
    +
    +			InetAddress taskManagerAddress = LeaderRetrievalUtils.findConnectingAddress(leaderRetrievalService, lookupTimeout);
    +			taskManagerHostname = taskManagerAddress.getHostName();
    +			LOG.info("TaskManager will use hostname/address '{}' ({}) for communication.",
    +				taskManagerHostname, taskManagerAddress.getHostAddress());
    +		}
    +
    +		// if no task manager port has been configured, use 0 (system will pick any free port)
    +		int actorSystemPort = configuration.getInteger(ConfigConstants.TASK_MANAGER_IPC_PORT_KEY, 0);
    +		if (actorSystemPort < 0 || actorSystemPort > 65535) {
    +			throw new IllegalConfigurationException("Invalid value for '" +
    +				ConfigConstants.TASK_MANAGER_IPC_PORT_KEY +
    +				"' (port for the TaskManager actor system) : " + actorSystemPort +
    +				" - Leave config parameter empty or use 0 to let the system choose a port automatically.");
    +		}
    +
    +		return new Tuple2<>(taskManagerHostname, actorSystemPort);
    +	}
    +
    +	/**
    +	 * Starts and runs the TaskManager. Brings up an actor system for the TaskManager and its
    +	 * actors, starts the TaskManager's services (library cache, shuffle network stack, ...),
    +	 * and starts the TaskManager itself.
    +	 * <p/>
    +	 * This method will also spawn a process reaper for the TaskManager (kill the process if
    +	 * the actor fails) and optionally start the JVM memory logging thread.
    +	 *
    +	 * @param taskManagerHostname The hostname/address of the interface where the actor system
    +	 *                            will communicate.
    +	 * @param resourceID          The id of the resource which the task manager will run on.
    +	 * @param actorSystemPort     The port at which the actor system will communicate.
    +	 * @param configuration       The configuration for the TaskManager.
    +	 * @param taskManagerClass    The actor class to instantiate. Allows the use of TaskManager
    +	 *                            subclasses for example for YARN.
    +	 */
    +	private static void runTaskManager(
    +		String taskManagerHostname,
    +		ResourceID resourceID,
    +		int actorSystemPort,
    +		Configuration configuration,
    +		Class<? extends TaskManager> taskManagerClass) throws Exception {
    +
    +		LOG.info("Starting TaskManager");
    +
    +		// Bring up the TaskManager actor system first, bind it to the given address.
    +
    +		LOG.info("Starting TaskManager actor system at " +
    +			NetUtils.hostAndPortToUrlString(taskManagerHostname, actorSystemPort));
    +
    +		ActorSystem taskManagerSystem;
    +		try {
    +			Tuple2<String, Object> address = new Tuple2<String, Object>(taskManagerHostname, actorSystemPort);
    +			Config akkaConfig = AkkaUtils.getAkkaConfig(configuration, new Some<>(address));
    +			LOG.debug("Using akka configuration\n " + akkaConfig);
    +			taskManagerSystem = AkkaUtils.createActorSystem(akkaConfig);
    +		} catch (Throwable t) {
    +			if (t instanceof org.jboss.netty.channel.ChannelException) {
    +				Throwable cause = t.getCause();
    +				if (cause != null && t.getCause() instanceof java.net.BindException) {
    +					String address = NetUtils.hostAndPortToUrlString(taskManagerHostname, actorSystemPort);
    +					throw new IOException("Unable to bind TaskManager actor system to address " +
    +						address + " - " + cause.getMessage(), t);
    +				}
    +			}
    +			throw new Exception("Could not create TaskManager actor system", t);
    +		}
    +
    +		// start all the TaskManager services (network stack,  library cache, ...)
    +		// and the TaskManager actor
    +		try {
    +			LOG.info("Starting TaskManager actor");
    +			ActorRef taskManagerActor = startTaskManagerComponentsAndActor(
    +				configuration,
    +				resourceID,
    +				taskManagerSystem,
    +				taskManagerHostname,
    +				TASK_MANAGER_NAME,
    +				null,
    +				false,
    +				taskManagerClass);
    +
    +			// start a process reaper that watches the JobManager. If the TaskManager actor dies,
    +			// the process reaper will kill the JVM process (to ensure easy failure detection)
    +			LOG.debug("Starting TaskManager process reaper");
    +
    +			taskManagerSystem.actorOf(
    +				Props.create(ProcessReaper.class, taskManagerActor, LOG, RUNTIME_FAILURE_RETURN_CODE),
    +				"TaskManager_Process_Reaper");
    +
    +			// if desired, start the logging daemon that periodically logs the memory usage information
    +			if (LOG.isInfoEnabled() && configuration.getBoolean(
    +				ConfigConstants.TASK_MANAGER_DEBUG_MEMORY_USAGE_START_LOG_THREAD,
    +				ConfigConstants.DEFAULT_TASK_MANAGER_DEBUG_MEMORY_USAGE_START_LOG_THREAD)) {
    +				LOG.info("Starting periodic memory usage logger");
    +
    +				long interval = configuration.getLong(
    +					ConfigConstants.TASK_MANAGER_DEBUG_MEMORY_USAGE_LOG_INTERVAL_MS,
    +					ConfigConstants.DEFAULT_TASK_MANAGER_DEBUG_MEMORY_USAGE_LOG_INTERVAL_MS);
    +
    +				MemoryLogger logger = new MemoryLogger(LOG, interval, taskManagerSystem);
    +				logger.start();
    +			}
    +
    +			// block until everything is done
    +			taskManagerSystem.awaitTermination();
    +		} catch (Throwable t) {
    +			LOG.error("Error while starting up taskManager", t);
    +			try {
    +				taskManagerSystem.shutdown();
    +			} catch (Throwable tt) {
    +				LOG.warn("Could not cleanly shut down actor system", tt);
    +			}
    +			throw t;
    +		}
    +	}
    +
    +	// --------------------------------------------------------------------------
    +	//  Starting and running the TaskManager
    +	// --------------------------------------------------------------------------
    +
    +	/**
    +	 * @param configuration                 The configuration for the TaskManager.
    +	 * @param resourceID                    The id of the resource which the task manager will run on.
    +	 * @param actorSystem                  The actor system that should run the TaskManager actor.
    +	 * @param taskManagerHostname       The hostname/address that describes the TaskManager's data location.
    +	 * @param taskManagerActorName      Optionally the name of the TaskManager actor. If none is given,
    +	 *                                      the actor will use a random name.
    +	 * @param leaderRetrievalService        Optionally, a leader retrieval service can be provided. If none is given,
    +	 *                                      then a LeaderRetrievalService is constructed from the configuration.
    +	 * @param localTaskManagerCommunication If true, the TaskManager will not initiate the TCP network stack.
    +	 * @param taskManagerClass      The class of the TaskManager actor. May be used to give
    +	 *                                      subclasses that understand additional actor messages.
    +	 * @return An ActorRef to the TaskManager actor.
    +	 * @throws org.apache.flink.configuration.IllegalConfigurationException     Thrown, if the given config contains illegal values.
    +	 * @throws java.io.IOException      Thrown, if any of the I/O components (such as buffer pools,
    +	 *                                       I/O manager, ...) cannot be properly started.
    +	 * @throws java.lang.Exception      Thrown is some other error occurs while parsing the configuration
    +	 *                                      or starting the TaskManager components.
    +	 */
    +	public static ActorRef startTaskManagerComponentsAndActor(
    +		Configuration configuration,
    +		ResourceID resourceID,
    +		ActorSystem actorSystem,
    +		String taskManagerHostname,
    +		String taskManagerActorName,
    +		LeaderRetrievalService leaderRetrievalService,
    +		boolean localTaskManagerCommunication,
    +		Class<? extends TaskManager> taskManagerClass) throws Exception {
    +
    +		Tuple4<TaskManagerConfiguration, NetworkEnvironmentConfiguration, InstanceConnectionInfo, MemoryType> tuple4
    +			= parseTaskManagerConfiguration(configuration, taskManagerHostname, localTaskManagerCommunication);
    +
    +		TaskManagerConfiguration taskManagerConfig = tuple4._1();
    +		NetworkEnvironmentConfiguration netConfig = tuple4._2();
    +		InstanceConnectionInfo connectionInfo = tuple4._3();
    +		MemoryType memType = tuple4._4();
    +
    +		// pre-start checks
    +		checkTempDirs(taskManagerConfig.tmpDirPaths());
    +
    +		ExecutionContext executionContext = ExecutionContexts$.MODULE$.fromExecutor(new ForkJoinPool());
    +
    +		// we start the network first, to make sure it can allocate its buffers first
    +		NetworkEnvironment network = new NetworkEnvironment(executionContext, taskManagerConfig.timeout(), netConfig);
    +
    +		// computing the amount of memory to use depends on how much memory is available
    +		// it strictly needs to happen AFTER the network stack has been initialized
    +
    +		// check if a value has been configured
    +		long configuredMemory = configuration.getLong(ConfigConstants.TASK_MANAGER_MEMORY_SIZE_KEY, -1L);
    +		checkConfigParameter(configuredMemory == -1 || configuredMemory > 0, configuredMemory,
    +			ConfigConstants.TASK_MANAGER_MEMORY_SIZE_KEY,
    +			"MemoryManager needs at least one MB of memory. " +
    +				"If you leave this config parameter empty, the system automatically " +
    +				"pick a fraction of the available memory.");
    +
    +		long memorySize;
    +		boolean preAllocateMemory = configuration.getBoolean(
    +			ConfigConstants.TASK_MANAGER_MEMORY_PRE_ALLOCATE_KEY,
    +			ConfigConstants.DEFAULT_TASK_MANAGER_MEMORY_PRE_ALLOCATE);
    +		if (configuredMemory > 0) {
    +			if (preAllocateMemory) {
    +				LOG.info("Using {} MB for managed memory." , configuredMemory);
    +			} else {
    +				LOG.info("Limiting managed memory to {} MB, memory will be allocated lazily." , configuredMemory);
    +			}
    +			memorySize = configuredMemory << 20; // megabytes to bytes
    +		} else {
    +			float fraction = configuration.getFloat(
    +				ConfigConstants.TASK_MANAGER_MEMORY_FRACTION_KEY,
    +				ConfigConstants.DEFAULT_MEMORY_MANAGER_MEMORY_FRACTION);
    +			checkConfigParameter(fraction > 0.0f && fraction < 1.0f, fraction,
    +				ConfigConstants.TASK_MANAGER_MEMORY_FRACTION_KEY,
    +				"MemoryManager fraction of the free memory must be between 0.0 and 1.0");
    +
    +			if (memType == MemoryType.HEAP) {
    +				long relativeMemSize = (long) (EnvironmentInformation.getSizeOfFreeHeapMemoryWithDefrag() * fraction);
    +				if (preAllocateMemory) {
    +					LOG.info("Using {} of the currently free heap space for managed heap memory ({} MB)." ,
    +						fraction , relativeMemSize >> 20);
    +				} else {
    +					LOG.info("Limiting managed memory to {} of the currently free heap space ({} MB), " +
    +						"memory will be allocated lazily." , fraction , relativeMemSize >> 20);
    +				}
    +				memorySize = relativeMemSize;
    +			} else if (memType == MemoryType.OFF_HEAP) {
    +				// The maximum heap memory has been adjusted according to the fraction
    +				long maxMemory = EnvironmentInformation.getMaxJvmHeapMemory();
    +				long directMemorySize = (long) (maxMemory / (1.0 - fraction) * fraction);
    +				if (preAllocateMemory) {
    +					LOG.info("Using {} of the maximum memory size for managed off-heap memory ({} MB)." ,
    +						fraction, directMemorySize >> 20);
    +				} else {
    +					LOG.info("Limiting managed memory to {} of the maximum memory size ({} MB)," +
    +						" memory will be allocated lazily.", fraction, directMemorySize >> 20);
    +				}
    +				memorySize = directMemorySize;
    +			} else {
    +				throw new RuntimeException("No supported memory type detected.");
    +			}
    +		}
    +
    +		// now start the memory manager
    +		MemoryManager memoryManager;
    +		try {
    +			memoryManager = new MemoryManager(
    +				memorySize,
    +				taskManagerConfig.numberOfSlots(),
    +				netConfig.networkBufferSize(),
    +				memType,
    +				preAllocateMemory);
    +		} catch (OutOfMemoryError e) {
    +			if (memType == MemoryType.HEAP) {
    +				throw new Exception("OutOfMemory error (" + e.getMessage() +
    +					") while allocating the TaskManager heap memory (" + memorySize + " bytes).", e);
    +			} else if (memType == MemoryType.OFF_HEAP) {
    +				throw new Exception("OutOfMemory error (" + e.getMessage() +
    +					") while allocating the TaskManager off-heap memory (" + memorySize +
    +					" bytes).Try increasing the maximum direct memory (-XX:MaxDirectMemorySize)", e);
    +			} else {
    +				throw e;
    +			}
    +		}
    +
    +		// start the I/O manager, it will create some temp directories.
    +		IOManager ioManager = new IOManagerAsync(taskManagerConfig.tmpDirPaths());
    +
    +		if (leaderRetrievalService == null){
    +			leaderRetrievalService = LeaderRetrievalUtils.createLeaderRetrievalService(configuration);
    +		}
    +
    +		// create the actor properties (which define the actor constructor parameters)
    +		Props tmProps = Props.create(
    +			taskManagerClass,
    +			taskManagerConfig,
    +			resourceID,
    +			connectionInfo,
    +			memoryManager,
    +			ioManager,
    +			network,
    +			taskManagerConfig.numberOfSlots(),
    +			leaderRetrievalService);
    +
    +		ActorRef taskManagerActorRef;
    +		if (taskManagerActorName != null && !taskManagerActorName.equals("")) {
    +			taskManagerActorRef = actorSystem.actorOf(tmProps, taskManagerActorName);
    +		} else {
    +			taskManagerActorRef = actorSystem.actorOf(tmProps);
    +		}
    +
    +		return taskManagerActorRef;
    --- End diff --
    
    I think there should be no `ActorRef`s at this point. Instead the RpcService should be passed.


---
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.
---

[GitHub] flink pull request #2400: [FLINK-4363] Implement TaskManager basic startup o...

Posted by mxm <gi...@git.apache.org>.
Github user mxm commented on a diff in the pull request:

    https://github.com/apache/flink/pull/2400#discussion_r75860606
  
    --- Diff: flink-runtime/src/main/java/org/apache/flink/runtime/rpc/taskexecutor/TaskExecutor.java ---
    @@ -35,27 +79,634 @@
      */
     public class TaskExecutor extends RpcEndpoint<TaskExecutorGateway> {
     
    -	/** The unique resource ID of this TaskExecutor */
    +	private static final Logger LOG = LoggerFactory.getLogger(TaskExecutor.class);
    +
    +	/** Return code for critical errors during the runtime */
    +	private static final int RUNTIME_FAILURE_RETURN_CODE = 2;
    +
    +	/** The name of the TaskManager actor */
    +	private static final String TASK_MANAGER_NAME = "taskmanager";
    +
    +	/** The unique resource ID of this TaskManager */
     	private final ResourceID resourceID;
     
     	/** The access to the leader election and metadata storage services */
     	private final HighAvailabilityServices haServices;
     
    -	// --------- resource manager --------
    +	/** The task manager configuration */
    +	private final TaskManagerConfiguration taskManagerConfig;
    +
    +	/** The connection information of the task manager */
    +	private final InstanceConnectionInfo connectionInfo;
    +
    +	/** The I/O manager component in the task manager */
    +	private final IOManager ioManager;
    +
    +	/** The memory manager component in the task manager */
    +	private final MemoryManager memoryManager;
    +
    +	/** The network component in the task manager */
    +	private final NetworkEnvironment networkEnvironment;
    +
    +	/** The number of slots in the task manager, should be 1 for YARN */
    +	private final int numberOfSlots;
     
    +	// --------- resource manager --------
     	private TaskExecutorToResourceManagerConnection resourceManagerConnection;
     
     	// ------------------------------------------------------------------------
     
     	public TaskExecutor(
    +			TaskManagerConfiguration taskManagerConfig,
    +			ResourceID resourceID,
    +			InstanceConnectionInfo connectionInfo,
    +			MemoryManager memoryManager,
    +			IOManager ioManager,
    +			NetworkEnvironment networkEnvironment,
    +			int numberOfSlots,
     			RpcService rpcService,
    -			HighAvailabilityServices haServices,
    -			ResourceID resourceID) {
    +			HighAvailabilityServices haServices) {
     
     		super(rpcService);
     
    -		this.haServices = checkNotNull(haServices);
    +		this.taskManagerConfig = checkNotNull(taskManagerConfig);
     		this.resourceID = checkNotNull(resourceID);
    +		this.connectionInfo = checkNotNull(connectionInfo);
    +		this.memoryManager = checkNotNull(memoryManager);
    +		this.ioManager = checkNotNull(ioManager);
    +		this.networkEnvironment = checkNotNull(networkEnvironment);
    +		this.numberOfSlots = checkNotNull(numberOfSlots);
    +		this.haServices = checkNotNull(haServices);
    +	}
    +
    +	/**
    +	 * Starts and runs the TaskManager.
    +	 * <p/>
    +	 * This method first tries to select the network interface to use for the TaskManager
    +	 * communication. The network interface is used both for the actor communication
    +	 * (coordination) as well as for the data exchange between task managers. Unless
    +	 * the hostname/interface is explicitly configured in the configuration, this
    +	 * method will try out various interfaces and methods to connect to the JobManager
    +	 * and select the one where the connection attempt is successful.
    +	 * <p/>
    +	 * After selecting the network interface, this method brings up an actor system
    +	 * for the TaskManager and its actors, starts the TaskManager's services
    +	 * (library cache, shuffle network stack, ...), and starts the TaskManager itself.
    +	 *
    +	 * @param configuration    The configuration for the TaskManager.
    +	 * @param taskManagerClass The actor class to instantiate.
    +	 *                         Allows to use TaskManager subclasses for example for YARN.
    +	 */
    +	public static void selectNetworkInterfaceAndRunTaskManager(
    +		Configuration configuration,
    +		ResourceID resourceID,
    +		Class<? extends TaskManager> taskManagerClass) throws Exception {
    +
    +		Tuple2<String, Integer> tuple2 = selectNetworkInterfaceAndPort(configuration);
    +
    +		runTaskManager(tuple2._1(), resourceID, tuple2._2(), configuration, taskManagerClass);
    +	}
    +
    +	private static Tuple2<String, Integer> selectNetworkInterfaceAndPort(Configuration configuration)
    +		throws Exception {
    +		String taskManagerHostname = configuration.getString(ConfigConstants.TASK_MANAGER_HOSTNAME_KEY, null);
    +		if (taskManagerHostname != null) {
    +			LOG.info("Using configured hostname/address for TaskManager: " + taskManagerHostname);
    +		} else {
    +			LeaderRetrievalService leaderRetrievalService = LeaderRetrievalUtils.createLeaderRetrievalService(configuration);
    +			FiniteDuration lookupTimeout = AkkaUtils.getLookupTimeout(configuration);
    +
    +			InetAddress taskManagerAddress = LeaderRetrievalUtils.findConnectingAddress(leaderRetrievalService, lookupTimeout);
    +			taskManagerHostname = taskManagerAddress.getHostName();
    +			LOG.info("TaskManager will use hostname/address '{}' ({}) for communication.",
    +				taskManagerHostname, taskManagerAddress.getHostAddress());
    +		}
    +
    +		// if no task manager port has been configured, use 0 (system will pick any free port)
    +		int actorSystemPort = configuration.getInteger(ConfigConstants.TASK_MANAGER_IPC_PORT_KEY, 0);
    +		if (actorSystemPort < 0 || actorSystemPort > 65535) {
    +			throw new IllegalConfigurationException("Invalid value for '" +
    +				ConfigConstants.TASK_MANAGER_IPC_PORT_KEY +
    +				"' (port for the TaskManager actor system) : " + actorSystemPort +
    +				" - Leave config parameter empty or use 0 to let the system choose a port automatically.");
    +		}
    +
    +		return new Tuple2<>(taskManagerHostname, actorSystemPort);
    +	}
    +
    +	/**
    +	 * Starts and runs the TaskManager. Brings up an actor system for the TaskManager and its
    +	 * actors, starts the TaskManager's services (library cache, shuffle network stack, ...),
    +	 * and starts the TaskManager itself.
    +	 * <p/>
    +	 * This method will also spawn a process reaper for the TaskManager (kill the process if
    +	 * the actor fails) and optionally start the JVM memory logging thread.
    +	 *
    +	 * @param taskManagerHostname The hostname/address of the interface where the actor system
    +	 *                            will communicate.
    +	 * @param resourceID          The id of the resource which the task manager will run on.
    +	 * @param actorSystemPort     The port at which the actor system will communicate.
    +	 * @param configuration       The configuration for the TaskManager.
    +	 * @param taskManagerClass    The actor class to instantiate. Allows the use of TaskManager
    +	 *                            subclasses for example for YARN.
    +	 */
    +	private static void runTaskManager(
    +		String taskManagerHostname,
    +		ResourceID resourceID,
    +		int actorSystemPort,
    +		Configuration configuration,
    +		Class<? extends TaskManager> taskManagerClass) throws Exception {
    +
    +		LOG.info("Starting TaskManager");
    +
    +		// Bring up the TaskManager actor system first, bind it to the given address.
    +
    +		LOG.info("Starting TaskManager actor system at " +
    +			NetUtils.hostAndPortToUrlString(taskManagerHostname, actorSystemPort));
    +
    +		ActorSystem taskManagerSystem;
    +		try {
    +			Tuple2<String, Object> address = new Tuple2<String, Object>(taskManagerHostname, actorSystemPort);
    +			Config akkaConfig = AkkaUtils.getAkkaConfig(configuration, new Some<>(address));
    +			LOG.debug("Using akka configuration\n " + akkaConfig);
    +			taskManagerSystem = AkkaUtils.createActorSystem(akkaConfig);
    +		} catch (Throwable t) {
    +			if (t instanceof org.jboss.netty.channel.ChannelException) {
    +				Throwable cause = t.getCause();
    +				if (cause != null && t.getCause() instanceof java.net.BindException) {
    +					String address = NetUtils.hostAndPortToUrlString(taskManagerHostname, actorSystemPort);
    +					throw new IOException("Unable to bind TaskManager actor system to address " +
    +						address + " - " + cause.getMessage(), t);
    +				}
    +			}
    +			throw new Exception("Could not create TaskManager actor system", t);
    +		}
    +
    +		// start all the TaskManager services (network stack,  library cache, ...)
    +		// and the TaskManager actor
    +		try {
    +			LOG.info("Starting TaskManager actor");
    +			ActorRef taskManagerActor = startTaskManagerComponentsAndActor(
    +				configuration,
    +				resourceID,
    +				taskManagerSystem,
    +				taskManagerHostname,
    +				TASK_MANAGER_NAME,
    +				null,
    +				false,
    +				taskManagerClass);
    +
    +			// start a process reaper that watches the JobManager. If the TaskManager actor dies,
    +			// the process reaper will kill the JVM process (to ensure easy failure detection)
    +			LOG.debug("Starting TaskManager process reaper");
    +
    +			taskManagerSystem.actorOf(
    +				Props.create(ProcessReaper.class, taskManagerActor, LOG, RUNTIME_FAILURE_RETURN_CODE),
    +				"TaskManager_Process_Reaper");
    +
    +			// if desired, start the logging daemon that periodically logs the memory usage information
    +			if (LOG.isInfoEnabled() && configuration.getBoolean(
    +				ConfigConstants.TASK_MANAGER_DEBUG_MEMORY_USAGE_START_LOG_THREAD,
    +				ConfigConstants.DEFAULT_TASK_MANAGER_DEBUG_MEMORY_USAGE_START_LOG_THREAD)) {
    +				LOG.info("Starting periodic memory usage logger");
    +
    +				long interval = configuration.getLong(
    +					ConfigConstants.TASK_MANAGER_DEBUG_MEMORY_USAGE_LOG_INTERVAL_MS,
    +					ConfigConstants.DEFAULT_TASK_MANAGER_DEBUG_MEMORY_USAGE_LOG_INTERVAL_MS);
    +
    +				MemoryLogger logger = new MemoryLogger(LOG, interval, taskManagerSystem);
    +				logger.start();
    +			}
    +
    +			// block until everything is done
    +			taskManagerSystem.awaitTermination();
    +		} catch (Throwable t) {
    +			LOG.error("Error while starting up taskManager", t);
    +			try {
    +				taskManagerSystem.shutdown();
    +			} catch (Throwable tt) {
    +				LOG.warn("Could not cleanly shut down actor system", tt);
    +			}
    +			throw t;
    +		}
    +	}
    +
    +	// --------------------------------------------------------------------------
    +	//  Starting and running the TaskManager
    +	// --------------------------------------------------------------------------
    +
    +	/**
    +	 * @param configuration                 The configuration for the TaskManager.
    +	 * @param resourceID                    The id of the resource which the task manager will run on.
    +	 * @param actorSystem                  The actor system that should run the TaskManager actor.
    +	 * @param taskManagerHostname       The hostname/address that describes the TaskManager's data location.
    +	 * @param taskManagerActorName      Optionally the name of the TaskManager actor. If none is given,
    +	 *                                      the actor will use a random name.
    +	 * @param leaderRetrievalService        Optionally, a leader retrieval service can be provided. If none is given,
    +	 *                                      then a LeaderRetrievalService is constructed from the configuration.
    +	 * @param localTaskManagerCommunication If true, the TaskManager will not initiate the TCP network stack.
    +	 * @param taskManagerClass      The class of the TaskManager actor. May be used to give
    +	 *                                      subclasses that understand additional actor messages.
    +	 * @return An ActorRef to the TaskManager actor.
    +	 * @throws org.apache.flink.configuration.IllegalConfigurationException     Thrown, if the given config contains illegal values.
    +	 * @throws java.io.IOException      Thrown, if any of the I/O components (such as buffer pools,
    +	 *                                       I/O manager, ...) cannot be properly started.
    +	 * @throws java.lang.Exception      Thrown is some other error occurs while parsing the configuration
    +	 *                                      or starting the TaskManager components.
    +	 */
    +	public static ActorRef startTaskManagerComponentsAndActor(
    +		Configuration configuration,
    +		ResourceID resourceID,
    +		ActorSystem actorSystem,
    +		String taskManagerHostname,
    +		String taskManagerActorName,
    +		LeaderRetrievalService leaderRetrievalService,
    +		boolean localTaskManagerCommunication,
    +		Class<? extends TaskManager> taskManagerClass) throws Exception {
    +
    +		Tuple4<TaskManagerConfiguration, NetworkEnvironmentConfiguration, InstanceConnectionInfo, MemoryType> tuple4
    +			= parseTaskManagerConfiguration(configuration, taskManagerHostname, localTaskManagerCommunication);
    +
    +		TaskManagerConfiguration taskManagerConfig = tuple4._1();
    +		NetworkEnvironmentConfiguration netConfig = tuple4._2();
    +		InstanceConnectionInfo connectionInfo = tuple4._3();
    +		MemoryType memType = tuple4._4();
    +
    +		// pre-start checks
    +		checkTempDirs(taskManagerConfig.tmpDirPaths());
    +
    +		ExecutionContext executionContext = ExecutionContexts$.MODULE$.fromExecutor(new ForkJoinPool());
    +
    +		// we start the network first, to make sure it can allocate its buffers first
    +		NetworkEnvironment network = new NetworkEnvironment(executionContext, taskManagerConfig.timeout(), netConfig);
    +
    +		// computing the amount of memory to use depends on how much memory is available
    +		// it strictly needs to happen AFTER the network stack has been initialized
    +
    +		// check if a value has been configured
    +		long configuredMemory = configuration.getLong(ConfigConstants.TASK_MANAGER_MEMORY_SIZE_KEY, -1L);
    +		checkConfigParameter(configuredMemory == -1 || configuredMemory > 0, configuredMemory,
    +			ConfigConstants.TASK_MANAGER_MEMORY_SIZE_KEY,
    +			"MemoryManager needs at least one MB of memory. " +
    +				"If you leave this config parameter empty, the system automatically " +
    +				"pick a fraction of the available memory.");
    +
    +		long memorySize;
    +		boolean preAllocateMemory = configuration.getBoolean(
    +			ConfigConstants.TASK_MANAGER_MEMORY_PRE_ALLOCATE_KEY,
    +			ConfigConstants.DEFAULT_TASK_MANAGER_MEMORY_PRE_ALLOCATE);
    +		if (configuredMemory > 0) {
    +			if (preAllocateMemory) {
    +				LOG.info("Using {} MB for managed memory." , configuredMemory);
    +			} else {
    +				LOG.info("Limiting managed memory to {} MB, memory will be allocated lazily." , configuredMemory);
    +			}
    +			memorySize = configuredMemory << 20; // megabytes to bytes
    +		} else {
    +			float fraction = configuration.getFloat(
    +				ConfigConstants.TASK_MANAGER_MEMORY_FRACTION_KEY,
    +				ConfigConstants.DEFAULT_MEMORY_MANAGER_MEMORY_FRACTION);
    +			checkConfigParameter(fraction > 0.0f && fraction < 1.0f, fraction,
    +				ConfigConstants.TASK_MANAGER_MEMORY_FRACTION_KEY,
    +				"MemoryManager fraction of the free memory must be between 0.0 and 1.0");
    +
    +			if (memType == MemoryType.HEAP) {
    +				long relativeMemSize = (long) (EnvironmentInformation.getSizeOfFreeHeapMemoryWithDefrag() * fraction);
    +				if (preAllocateMemory) {
    +					LOG.info("Using {} of the currently free heap space for managed heap memory ({} MB)." ,
    +						fraction , relativeMemSize >> 20);
    +				} else {
    +					LOG.info("Limiting managed memory to {} of the currently free heap space ({} MB), " +
    +						"memory will be allocated lazily." , fraction , relativeMemSize >> 20);
    +				}
    +				memorySize = relativeMemSize;
    +			} else if (memType == MemoryType.OFF_HEAP) {
    +				// The maximum heap memory has been adjusted according to the fraction
    +				long maxMemory = EnvironmentInformation.getMaxJvmHeapMemory();
    +				long directMemorySize = (long) (maxMemory / (1.0 - fraction) * fraction);
    +				if (preAllocateMemory) {
    +					LOG.info("Using {} of the maximum memory size for managed off-heap memory ({} MB)." ,
    +						fraction, directMemorySize >> 20);
    +				} else {
    +					LOG.info("Limiting managed memory to {} of the maximum memory size ({} MB)," +
    +						" memory will be allocated lazily.", fraction, directMemorySize >> 20);
    +				}
    +				memorySize = directMemorySize;
    +			} else {
    +				throw new RuntimeException("No supported memory type detected.");
    +			}
    +		}
    +
    +		// now start the memory manager
    +		MemoryManager memoryManager;
    +		try {
    +			memoryManager = new MemoryManager(
    +				memorySize,
    +				taskManagerConfig.numberOfSlots(),
    +				netConfig.networkBufferSize(),
    +				memType,
    +				preAllocateMemory);
    +		} catch (OutOfMemoryError e) {
    +			if (memType == MemoryType.HEAP) {
    +				throw new Exception("OutOfMemory error (" + e.getMessage() +
    +					") while allocating the TaskManager heap memory (" + memorySize + " bytes).", e);
    +			} else if (memType == MemoryType.OFF_HEAP) {
    +				throw new Exception("OutOfMemory error (" + e.getMessage() +
    +					") while allocating the TaskManager off-heap memory (" + memorySize +
    +					" bytes).Try increasing the maximum direct memory (-XX:MaxDirectMemorySize)", e);
    +			} else {
    +				throw e;
    +			}
    +		}
    +
    +		// start the I/O manager, it will create some temp directories.
    +		IOManager ioManager = new IOManagerAsync(taskManagerConfig.tmpDirPaths());
    +
    +		if (leaderRetrievalService == null){
    +			leaderRetrievalService = LeaderRetrievalUtils.createLeaderRetrievalService(configuration);
    +		}
    +
    +		// create the actor properties (which define the actor constructor parameters)
    +		Props tmProps = Props.create(
    +			taskManagerClass,
    +			taskManagerConfig,
    +			resourceID,
    +			connectionInfo,
    +			memoryManager,
    +			ioManager,
    +			network,
    +			taskManagerConfig.numberOfSlots(),
    +			leaderRetrievalService);
    +
    +		ActorRef taskManagerActorRef;
    +		if (taskManagerActorName != null && !taskManagerActorName.equals("")) {
    +			taskManagerActorRef = actorSystem.actorOf(tmProps, taskManagerActorName);
    +		} else {
    +			taskManagerActorRef = actorSystem.actorOf(tmProps);
    +		}
    +
    +		return taskManagerActorRef;
    --- End diff --
    
    The `HighAvailabilityService` is just an interface that you implement in a class which you pass to the constructor of the TaskExcutor. For now, the interface is sufficient because you only need to retrieve the ResourceManager to register. Let's assume the ResourceManager runs in the same Actor system as the JobManager until we have an additional leader retrieval for the resource manager.


---
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.
---

[GitHub] flink issue #2400: [FLINK-4363] Implement TaskManager basic startup of all c...

Posted by mxm <gi...@git.apache.org>.
Github user mxm commented on the issue:

    https://github.com/apache/flink/pull/2400
  
    Thank you @wangzhijiang999 for the changes. I've made a couple of more requests after which we can merge the PR.


---
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.
---

[GitHub] flink pull request #2400: [FLINK-4363] Implement TaskManager basic startup o...

Posted by wangzhijiang999 <gi...@git.apache.org>.
Github user wangzhijiang999 commented on a diff in the pull request:

    https://github.com/apache/flink/pull/2400#discussion_r76189334
  
    --- Diff: flink-runtime/src/main/java/org/apache/flink/runtime/rpc/taskexecutor/TaskExecutor.java ---
    @@ -94,12 +729,11 @@ public void notifyOfNewResourceManagerLeader(String newLeaderAddress, UUID newLe
     			if (newLeaderAddress != null) {
     				// the resource manager switched to a new leader
     				log.info("ResourceManager leader changed from {} to {}. Registering at new leader.",
    -						resourceManagerConnection.getResourceManagerAddress(), newLeaderAddress);
    --- End diff --
    
    The RPC methods and start method have been put at the top of the class after TaskExecutor constructor.


---
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.
---

[GitHub] flink pull request #2400: [FLINK-4363] Implement TaskManager basic startup o...

Posted by mxm <gi...@git.apache.org>.
Github user mxm commented on a diff in the pull request:

    https://github.com/apache/flink/pull/2400#discussion_r76084797
  
    --- Diff: flink-runtime/src/main/java/org/apache/flink/runtime/rpc/taskexecutor/TaskExecutor.java ---
    @@ -36,27 +82,617 @@
      */
     public class TaskExecutor extends RpcEndpoint<TaskExecutorGateway> {
     
    -	/** The unique resource ID of this TaskExecutor */
    +	private static final Logger LOG = LoggerFactory.getLogger(TaskExecutor.class);
    +
    +	/** The unique resource ID of this TaskManager */
     	private final ResourceID resourceID;
     
     	/** The access to the leader election and metadata storage services */
     	private final HighAvailabilityServices haServices;
     
    -	// --------- resource manager --------
    +	/** The task manager configuration */
    +	private final TaskExecutorConfiguration taskExecutorConfig;
    +
    +	/** The connection information of the task manager */
    +	private final InstanceConnectionInfo connectionInfo;
    +
    +	/** The I/O manager component in the task manager */
    +	private final IOManager ioManager;
    +
    +	/** The memory manager component in the task manager */
    +	private final MemoryManager memoryManager;
    +
    +	/** The network component in the task manager */
    +	private final NetworkEnvironment networkEnvironment;
    +
    +	/** The number of slots in the task manager, should be 1 for YARN */
    +	private final int numberOfSlots;
     
    +	// --------- resource manager --------
     	private TaskExecutorToResourceManagerConnection resourceManagerConnection;
     
     	// ------------------------------------------------------------------------
     
     	public TaskExecutor(
    +			TaskExecutorConfiguration taskExecutorConfig,
    +			ResourceID resourceID,
    +			InstanceConnectionInfo connectionInfo,
    +			MemoryManager memoryManager,
    +			IOManager ioManager,
    +			NetworkEnvironment networkEnvironment,
    +			int numberOfSlots,
     			RpcService rpcService,
    -			HighAvailabilityServices haServices,
    -			ResourceID resourceID) {
    +			HighAvailabilityServices haServices) {
     
     		super(rpcService);
     
    -		this.haServices = checkNotNull(haServices);
    +		this.taskExecutorConfig = checkNotNull(taskExecutorConfig);
     		this.resourceID = checkNotNull(resourceID);
    +		this.connectionInfo = checkNotNull(connectionInfo);
    +		this.memoryManager = checkNotNull(memoryManager);
    +		this.ioManager = checkNotNull(ioManager);
    +		this.networkEnvironment = checkNotNull(networkEnvironment);
    +		this.numberOfSlots = checkNotNull(numberOfSlots);
    +		this.haServices = checkNotNull(haServices);
    +	}
    +
    +	/**
    +	 * Starts and runs the TaskManager.
    +	 * <p/>
    +	 * This method first tries to select the network interface to use for the TaskManager
    +	 * communication. The network interface is used both for the actor communication
    +	 * (coordination) as well as for the data exchange between task managers. Unless
    +	 * the hostname/interface is explicitly configured in the configuration, this
    +	 * method will try out various interfaces and methods to connect to the JobManager
    +	 * and select the one where the connection attempt is successful.
    +	 * <p/>
    +	 * After selecting the network interface, this method brings up an actor system
    +	 * for the TaskManager and its actors, starts the TaskManager's services
    +	 * (library cache, shuffle network stack, ...), and starts the TaskManager itself.
    +	 *
    +	 * @param configuration    The configuration for the TaskManager.
    +	 * @param resourceID       The id of the resource which the task manager will run on.
    +	 */
    +	public static void selectNetworkInterfaceAndRunTaskManager(
    +		Configuration configuration,
    +		ResourceID resourceID) throws Exception {
    +
    +		InetSocketAddress taskManagerAddress = selectNetworkInterfaceAndPort(configuration);
    +
    +		runTaskManager(taskManagerAddress.getHostName(), resourceID, taskManagerAddress.getPort(), configuration);
    +	}
    +
    +	private static InetSocketAddress selectNetworkInterfaceAndPort(Configuration configuration)
    +		throws Exception {
    +		String taskManagerHostname = configuration.getString(ConfigConstants.TASK_MANAGER_HOSTNAME_KEY, null);
    +		if (taskManagerHostname != null) {
    +			LOG.info("Using configured hostname/address for TaskManager: " + taskManagerHostname);
    +		} else {
    +			LeaderRetrievalService leaderRetrievalService = LeaderRetrievalUtils.createLeaderRetrievalService(configuration);
    +			FiniteDuration lookupTimeout = AkkaUtils.getLookupTimeout(configuration);
    +
    +			InetAddress taskManagerAddress = LeaderRetrievalUtils.findConnectingAddress(leaderRetrievalService, lookupTimeout);
    +			taskManagerHostname = taskManagerAddress.getHostName();
    +			LOG.info("TaskManager will use hostname/address '{}' ({}) for communication.",
    +				taskManagerHostname, taskManagerAddress.getHostAddress());
    +		}
    +
    +		// if no task manager port has been configured, use 0 (system will pick any free port)
    +		int actorSystemPort = configuration.getInteger(ConfigConstants.TASK_MANAGER_IPC_PORT_KEY, 0);
    +		if (actorSystemPort < 0 || actorSystemPort > 65535) {
    +			throw new IllegalConfigurationException("Invalid value for '" +
    +				ConfigConstants.TASK_MANAGER_IPC_PORT_KEY +
    +				"' (port for the TaskManager actor system) : " + actorSystemPort +
    +				" - Leave config parameter empty or use 0 to let the system choose a port automatically.");
    +		}
    +
    +		return new InetSocketAddress(taskManagerHostname, actorSystemPort);
    +	}
    +
    +	/**
    +	 * Starts and runs the TaskManager. Brings up an actor system for the TaskManager and its
    +	 * actors, starts the TaskManager's services (library cache, shuffle network stack, ...),
    +	 * and starts the TaskManager itself.
    +	 * <p/>
    +	 * This method will also spawn a process reaper for the TaskManager (kill the process if
    +	 * the actor fails) and optionally start the JVM memory logging thread.
    +	 *
    +	 * @param taskManagerHostname The hostname/address of the interface where the actor system
    +	 *                            will communicate.
    +	 * @param resourceID          The id of the resource which the task manager will run on.
    +	 * @param actorSystemPort   The port at which the actor system will communicate.
    +	 * @param configuration       The configuration for the TaskManager.
    +	 */
    +	private static void runTaskManager(
    +		String taskManagerHostname,
    +		ResourceID resourceID,
    +		int actorSystemPort,
    +		final Configuration configuration) throws Exception {
    +
    +		LOG.info("Starting TaskManager");
    +
    +		// Bring up the TaskManager actor system first, bind it to the given address.
    +
    +		LOG.info("Starting TaskManager actor system at " +
    +			NetUtils.hostAndPortToUrlString(taskManagerHostname, actorSystemPort));
    +
    +		final ActorSystem taskManagerSystem;
    +		try {
    +			Tuple2<String, Object> address = new Tuple2<String, Object>(taskManagerHostname, actorSystemPort);
    +			Config akkaConfig = AkkaUtils.getAkkaConfig(configuration, new Some<>(address));
    +			LOG.debug("Using akka configuration\n " + akkaConfig);
    +			taskManagerSystem = AkkaUtils.createActorSystem(akkaConfig);
    +		} catch (Throwable t) {
    +			if (t instanceof org.jboss.netty.channel.ChannelException) {
    +				Throwable cause = t.getCause();
    +				if (cause != null && t.getCause() instanceof java.net.BindException) {
    +					String address = NetUtils.hostAndPortToUrlString(taskManagerHostname, actorSystemPort);
    +					throw new IOException("Unable to bind TaskManager actor system to address " +
    +						address + " - " + cause.getMessage(), t);
    +				}
    +			}
    +			throw new Exception("Could not create TaskManager actor system", t);
    +		}
    +
    +		// start akka rpc service based on actor system
    +		Timeout timeout = new Timeout(AkkaUtils.getTimeout(configuration).toMillis(), TimeUnit.MILLISECONDS);
    +		AkkaRpcService akkaRpcService = new AkkaRpcService(taskManagerSystem, timeout);
    +
    +		// start high availability service to implement getResourceManagerLeaderRetriever method only
    +		HighAvailabilityServices haServices = new HighAvailabilityServices() {
    +			@Override
    +			public LeaderRetrievalService getResourceManagerLeaderRetriever() throws Exception {
    +				return LeaderRetrievalUtils.createLeaderRetrievalService(configuration);
    +			}
    +
    +			@Override
    +			public LeaderElectionService getJobMasterLeaderElectionService(JobID jobID) throws Exception {
    +				return null;
    +			}
    +		};
    +
    +		// start all the TaskManager services (network stack,  library cache, ...)
    +		// and the TaskManager actor
    +		try {
    +			LOG.info("Starting TaskManager actor");
    +			TaskExecutor taskExecutor = startTaskManagerComponentsAndActor(
    +				configuration,
    +				resourceID,
    +				akkaRpcService,
    +				taskManagerHostname,
    +				haServices,
    +				false);
    +
    +			taskExecutor.start();
    +
    +			// if desired, start the logging daemon that periodically logs the memory usage information
    +			if (LOG.isInfoEnabled() && configuration.getBoolean(
    +				ConfigConstants.TASK_MANAGER_DEBUG_MEMORY_USAGE_START_LOG_THREAD,
    +				ConfigConstants.DEFAULT_TASK_MANAGER_DEBUG_MEMORY_USAGE_START_LOG_THREAD)) {
    +				LOG.info("Starting periodic memory usage logger");
    +
    +				long interval = configuration.getLong(
    +					ConfigConstants.TASK_MANAGER_DEBUG_MEMORY_USAGE_LOG_INTERVAL_MS,
    +					ConfigConstants.DEFAULT_TASK_MANAGER_DEBUG_MEMORY_USAGE_LOG_INTERVAL_MS);
    +
    +				MemoryLogger logger = new MemoryLogger(LOG, interval, taskManagerSystem);
    +				logger.start();
    +			}
    +
    +			// block until everything is done
    +			taskManagerSystem.awaitTermination();
    +		} catch (Throwable t) {
    +			LOG.error("Error while starting up taskManager", t);
    +			try {
    +				taskManagerSystem.shutdown();
    +			} catch (Throwable tt) {
    +				LOG.warn("Could not cleanly shut down actor system", tt);
    +			}
    +			throw t;
    +		}
    +	}
    +
    +	// --------------------------------------------------------------------------
    +	//  Starting and running the TaskManager
    +	// --------------------------------------------------------------------------
    +
    +	/**
    +	 * @param configuration                 The configuration for the TaskManager.
    +	 * @param resourceID                    The id of the resource which the task manager will run on.
    +	 * @param rpcService                  The rpc service which is used to start and connect to the TaskManager RpcEndpoint .
    +	 * @param taskManagerHostname       The hostname/address that describes the TaskManager's data location.
    +	 * @param haServices        Optionally, a high availability service can be provided. If none is given,
    +	 *                                      then a HighAvailabilityServices is constructed from the configuration.
    +	 * @param localTaskManagerCommunication If true, the TaskManager will not initiate the TCP network stack.
    +	 * @return An ActorRef to the TaskManager actor.
    +	 * @throws org.apache.flink.configuration.IllegalConfigurationException     Thrown, if the given config contains illegal values.
    +	 * @throws java.io.IOException      Thrown, if any of the I/O components (such as buffer pools,
    +	 *                                       I/O manager, ...) cannot be properly started.
    +	 * @throws java.lang.Exception      Thrown is some other error occurs while parsing the configuration
    +	 *                                      or starting the TaskManager components.
    +	 */
    +	public static TaskExecutor startTaskManagerComponentsAndActor(
    +		Configuration configuration,
    +		ResourceID resourceID,
    +		RpcService rpcService,
    +		String taskManagerHostname,
    +		HighAvailabilityServices haServices,
    +		boolean localTaskManagerCommunication) throws Exception {
    +
    +		Tuple4<TaskExecutorConfiguration, NetworkEnvironmentConfiguration, InstanceConnectionInfo, MemoryType> tuple4
    +			= parseTaskManagerConfiguration(configuration, taskManagerHostname, localTaskManagerCommunication);
    +
    +		TaskExecutorConfiguration taskExecutorConfig = tuple4.f0;
    +		NetworkEnvironmentConfiguration netConfig = tuple4.f1;
    +		InstanceConnectionInfo connectionInfo = tuple4.f2;
    +		MemoryType memType = tuple4.f3;
    +
    +		// pre-start checks
    +		checkTempDirs(taskExecutorConfig.getTmpDirPaths());
    +
    +		ExecutionContext executionContext = ExecutionContexts$.MODULE$.fromExecutor(new ForkJoinPool());
    +
    +		// we start the network first, to make sure it can allocate its buffers first
    +		NetworkEnvironment network = new NetworkEnvironment(executionContext, taskExecutorConfig.getTimeout(), netConfig);
    +
    +		// computing the amount of memory to use depends on how much memory is available
    +		// it strictly needs to happen AFTER the network stack has been initialized
    +
    +		// check if a value has been configured
    +		long configuredMemory = configuration.getLong(ConfigConstants.TASK_MANAGER_MEMORY_SIZE_KEY, -1L);
    +		checkConfigParameter(configuredMemory == -1 || configuredMemory > 0, configuredMemory,
    +			ConfigConstants.TASK_MANAGER_MEMORY_SIZE_KEY,
    +			"MemoryManager needs at least one MB of memory. " +
    +				"If you leave this config parameter empty, the system automatically " +
    +				"pick a fraction of the available memory.");
    +
    +		long memorySize;
    +		boolean preAllocateMemory = configuration.getBoolean(
    +			ConfigConstants.TASK_MANAGER_MEMORY_PRE_ALLOCATE_KEY,
    +			ConfigConstants.DEFAULT_TASK_MANAGER_MEMORY_PRE_ALLOCATE);
    +		if (configuredMemory > 0) {
    +			if (preAllocateMemory) {
    +				LOG.info("Using {} MB for managed memory." , configuredMemory);
    +			} else {
    +				LOG.info("Limiting managed memory to {} MB, memory will be allocated lazily." , configuredMemory);
    +			}
    +			memorySize = configuredMemory << 20; // megabytes to bytes
    +		} else {
    +			float fraction = configuration.getFloat(
    +				ConfigConstants.TASK_MANAGER_MEMORY_FRACTION_KEY,
    +				ConfigConstants.DEFAULT_MEMORY_MANAGER_MEMORY_FRACTION);
    +			checkConfigParameter(fraction > 0.0f && fraction < 1.0f, fraction,
    +				ConfigConstants.TASK_MANAGER_MEMORY_FRACTION_KEY,
    +				"MemoryManager fraction of the free memory must be between 0.0 and 1.0");
    +
    +			if (memType == MemoryType.HEAP) {
    +				long relativeMemSize = (long) (EnvironmentInformation.getSizeOfFreeHeapMemoryWithDefrag() * fraction);
    +				if (preAllocateMemory) {
    +					LOG.info("Using {} of the currently free heap space for managed heap memory ({} MB)." ,
    +						fraction , relativeMemSize >> 20);
    +				} else {
    +					LOG.info("Limiting managed memory to {} of the currently free heap space ({} MB), " +
    +						"memory will be allocated lazily." , fraction , relativeMemSize >> 20);
    +				}
    +				memorySize = relativeMemSize;
    +			} else if (memType == MemoryType.OFF_HEAP) {
    +				// The maximum heap memory has been adjusted according to the fraction
    +				long maxMemory = EnvironmentInformation.getMaxJvmHeapMemory();
    +				long directMemorySize = (long) (maxMemory / (1.0 - fraction) * fraction);
    +				if (preAllocateMemory) {
    +					LOG.info("Using {} of the maximum memory size for managed off-heap memory ({} MB)." ,
    +						fraction, directMemorySize >> 20);
    +				} else {
    +					LOG.info("Limiting managed memory to {} of the maximum memory size ({} MB)," +
    +						" memory will be allocated lazily.", fraction, directMemorySize >> 20);
    +				}
    +				memorySize = directMemorySize;
    +			} else {
    +				throw new RuntimeException("No supported memory type detected.");
    +			}
    +		}
    +
    +		// now start the memory manager
    +		final MemoryManager memoryManager;
    +		try {
    +			memoryManager = new MemoryManager(
    +				memorySize,
    +				taskExecutorConfig.getNumberOfSlots(),
    +				netConfig.networkBufferSize(),
    +				memType,
    +				preAllocateMemory);
    +		} catch (OutOfMemoryError e) {
    +			if (memType == MemoryType.HEAP) {
    +				throw new Exception("OutOfMemory error (" + e.getMessage() +
    +					") while allocating the TaskManager heap memory (" + memorySize + " bytes).", e);
    +			} else if (memType == MemoryType.OFF_HEAP) {
    +				throw new Exception("OutOfMemory error (" + e.getMessage() +
    +					") while allocating the TaskManager off-heap memory (" + memorySize +
    +					" bytes).Try increasing the maximum direct memory (-XX:MaxDirectMemorySize)", e);
    +			} else {
    +				throw e;
    +			}
    +		}
    +
    +		// start the I/O manager, it will create some temp directories.
    +		IOManager ioManager = new IOManagerAsync(taskExecutorConfig.getTmpDirPaths());
    +
    +		TaskExecutor taskExecutor = new TaskExecutor(
    +			taskExecutorConfig,
    +			resourceID,
    +			connectionInfo,
    +			memoryManager,
    +			ioManager,
    +			network,
    +			taskExecutorConfig.getNumberOfSlots(),
    +			rpcService,
    +			haServices);
    +
    +		return taskExecutor;
    +	}
    +
    +	// --------------------------------------------------------------------------
    +	//  Parsing and checking the TaskManager Configuration
    +	// --------------------------------------------------------------------------
    +
    +	/**
    +	 * Utility method to extract TaskManager config parameters from the configuration and to
    +	 * sanity check them.
    +	 *
    +	 * @param configuration                 The configuration.
    +	 * @param taskManagerHostname           The host name under which the TaskManager communicates.
    +	 * @param localTaskManagerCommunication True, to skip initializing the network stack.
    +	 *                                      Use only in cases where only one task manager runs.
    +	 * @return A tuple (TaskManagerConfiguration, network configuration,
    +	 * InstanceConnectionInfo, JobManager actor Akka URL).
    +	 */
    +	private static Tuple4<TaskExecutorConfiguration, NetworkEnvironmentConfiguration, InstanceConnectionInfo, MemoryType>
    --- End diff --
    
    Could you pack these values into a wrapper? For example, name it `TaskExecutorConfiguration`. It would have the other values as fields.


---
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.
---

[GitHub] flink issue #2400: [FLINK-4363] Implement TaskManager basic startup of all c...

Posted by wangzhijiang999 <gi...@git.apache.org>.
Github user wangzhijiang999 commented on the issue:

    https://github.com/apache/flink/pull/2400
  
    @mxm , I have created the FLINK-4505 for TaskExecutorFactory issue. As you mentioned, it should be an abstract class and provide an abstract method maybe called 'createAndStartTaskExecutor()'. There may be at least three different specific factories(testing, yarn, standalone) extend TaskExecutorFactory to implement the method 'createAndStartTaskExecutor'. The parameters in constructor of specific factories are different based on different modes. For example: for StandaloneTaskExecutorFactory, the constructor parameter should be (Configuration configuration,ResourceID resourceID,
    RpcService rpcService,String taskManagerHostname,HighAvailabilityServices haServices,boolean localTaskManagerCommunication), and in the 'createAndStartTaskExecutor()' method it can invoke \u2018startTaskManagerComponentsAndActor' method in TaskExecutor to bring up TaskExecutor. Do you have any other advices, then I can start this subtask later.
    https://issues.apache.org/jira/browse/FLINK-4505


---
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.
---

[GitHub] flink pull request #2400: [FLINK-4363] Implement TaskManager basic startup o...

Posted by wangzhijiang999 <gi...@git.apache.org>.
Github user wangzhijiang999 commented on a diff in the pull request:

    https://github.com/apache/flink/pull/2400#discussion_r75650354
  
    --- Diff: flink-runtime/src/main/java/org/apache/flink/runtime/rpc/taskexecutor/TaskExecutor.java ---
    @@ -35,27 +79,634 @@
      */
     public class TaskExecutor extends RpcEndpoint<TaskExecutorGateway> {
     
    -	/** The unique resource ID of this TaskExecutor */
    +	private static final Logger LOG = LoggerFactory.getLogger(TaskExecutor.class);
    +
    +	/** Return code for critical errors during the runtime */
    +	private static final int RUNTIME_FAILURE_RETURN_CODE = 2;
    +
    +	/** The name of the TaskManager actor */
    +	private static final String TASK_MANAGER_NAME = "taskmanager";
    +
    +	/** The unique resource ID of this TaskManager */
     	private final ResourceID resourceID;
     
     	/** The access to the leader election and metadata storage services */
     	private final HighAvailabilityServices haServices;
     
    -	// --------- resource manager --------
    +	/** The task manager configuration */
    +	private final TaskManagerConfiguration taskManagerConfig;
    +
    +	/** The connection information of the task manager */
    +	private final InstanceConnectionInfo connectionInfo;
    +
    +	/** The I/O manager component in the task manager */
    +	private final IOManager ioManager;
    +
    +	/** The memory manager component in the task manager */
    +	private final MemoryManager memoryManager;
    +
    +	/** The network component in the task manager */
    +	private final NetworkEnvironment networkEnvironment;
    +
    +	/** The number of slots in the task manager, should be 1 for YARN */
    +	private final int numberOfSlots;
     
    +	// --------- resource manager --------
     	private TaskExecutorToResourceManagerConnection resourceManagerConnection;
     
     	// ------------------------------------------------------------------------
     
     	public TaskExecutor(
    +			TaskManagerConfiguration taskManagerConfig,
    +			ResourceID resourceID,
    +			InstanceConnectionInfo connectionInfo,
    +			MemoryManager memoryManager,
    +			IOManager ioManager,
    +			NetworkEnvironment networkEnvironment,
    +			int numberOfSlots,
     			RpcService rpcService,
    -			HighAvailabilityServices haServices,
    -			ResourceID resourceID) {
    +			HighAvailabilityServices haServices) {
     
     		super(rpcService);
     
    -		this.haServices = checkNotNull(haServices);
    +		this.taskManagerConfig = checkNotNull(taskManagerConfig);
     		this.resourceID = checkNotNull(resourceID);
    +		this.connectionInfo = checkNotNull(connectionInfo);
    +		this.memoryManager = checkNotNull(memoryManager);
    +		this.ioManager = checkNotNull(ioManager);
    +		this.networkEnvironment = checkNotNull(networkEnvironment);
    +		this.numberOfSlots = checkNotNull(numberOfSlots);
    +		this.haServices = checkNotNull(haServices);
    +	}
    +
    +	/**
    +	 * Starts and runs the TaskManager.
    +	 * <p/>
    +	 * This method first tries to select the network interface to use for the TaskManager
    +	 * communication. The network interface is used both for the actor communication
    +	 * (coordination) as well as for the data exchange between task managers. Unless
    +	 * the hostname/interface is explicitly configured in the configuration, this
    +	 * method will try out various interfaces and methods to connect to the JobManager
    +	 * and select the one where the connection attempt is successful.
    +	 * <p/>
    +	 * After selecting the network interface, this method brings up an actor system
    +	 * for the TaskManager and its actors, starts the TaskManager's services
    +	 * (library cache, shuffle network stack, ...), and starts the TaskManager itself.
    +	 *
    +	 * @param configuration    The configuration for the TaskManager.
    +	 * @param taskManagerClass The actor class to instantiate.
    +	 *                         Allows to use TaskManager subclasses for example for YARN.
    +	 */
    +	public static void selectNetworkInterfaceAndRunTaskManager(
    +		Configuration configuration,
    +		ResourceID resourceID,
    +		Class<? extends TaskManager> taskManagerClass) throws Exception {
    +
    +		Tuple2<String, Integer> tuple2 = selectNetworkInterfaceAndPort(configuration);
    +
    +		runTaskManager(tuple2._1(), resourceID, tuple2._2(), configuration, taskManagerClass);
    +	}
    +
    +	private static Tuple2<String, Integer> selectNetworkInterfaceAndPort(Configuration configuration)
    +		throws Exception {
    +		String taskManagerHostname = configuration.getString(ConfigConstants.TASK_MANAGER_HOSTNAME_KEY, null);
    +		if (taskManagerHostname != null) {
    +			LOG.info("Using configured hostname/address for TaskManager: " + taskManagerHostname);
    +		} else {
    +			LeaderRetrievalService leaderRetrievalService = LeaderRetrievalUtils.createLeaderRetrievalService(configuration);
    +			FiniteDuration lookupTimeout = AkkaUtils.getLookupTimeout(configuration);
    +
    +			InetAddress taskManagerAddress = LeaderRetrievalUtils.findConnectingAddress(leaderRetrievalService, lookupTimeout);
    +			taskManagerHostname = taskManagerAddress.getHostName();
    +			LOG.info("TaskManager will use hostname/address '{}' ({}) for communication.",
    +				taskManagerHostname, taskManagerAddress.getHostAddress());
    +		}
    +
    +		// if no task manager port has been configured, use 0 (system will pick any free port)
    +		int actorSystemPort = configuration.getInteger(ConfigConstants.TASK_MANAGER_IPC_PORT_KEY, 0);
    +		if (actorSystemPort < 0 || actorSystemPort > 65535) {
    +			throw new IllegalConfigurationException("Invalid value for '" +
    +				ConfigConstants.TASK_MANAGER_IPC_PORT_KEY +
    +				"' (port for the TaskManager actor system) : " + actorSystemPort +
    +				" - Leave config parameter empty or use 0 to let the system choose a port automatically.");
    +		}
    +
    +		return new Tuple2<>(taskManagerHostname, actorSystemPort);
    +	}
    +
    +	/**
    +	 * Starts and runs the TaskManager. Brings up an actor system for the TaskManager and its
    +	 * actors, starts the TaskManager's services (library cache, shuffle network stack, ...),
    +	 * and starts the TaskManager itself.
    +	 * <p/>
    +	 * This method will also spawn a process reaper for the TaskManager (kill the process if
    +	 * the actor fails) and optionally start the JVM memory logging thread.
    +	 *
    +	 * @param taskManagerHostname The hostname/address of the interface where the actor system
    +	 *                            will communicate.
    +	 * @param resourceID          The id of the resource which the task manager will run on.
    +	 * @param actorSystemPort     The port at which the actor system will communicate.
    +	 * @param configuration       The configuration for the TaskManager.
    +	 * @param taskManagerClass    The actor class to instantiate. Allows the use of TaskManager
    +	 *                            subclasses for example for YARN.
    +	 */
    +	private static void runTaskManager(
    +		String taskManagerHostname,
    +		ResourceID resourceID,
    +		int actorSystemPort,
    +		Configuration configuration,
    +		Class<? extends TaskManager> taskManagerClass) throws Exception {
    +
    +		LOG.info("Starting TaskManager");
    +
    +		// Bring up the TaskManager actor system first, bind it to the given address.
    +
    +		LOG.info("Starting TaskManager actor system at " +
    +			NetUtils.hostAndPortToUrlString(taskManagerHostname, actorSystemPort));
    +
    +		ActorSystem taskManagerSystem;
    +		try {
    +			Tuple2<String, Object> address = new Tuple2<String, Object>(taskManagerHostname, actorSystemPort);
    +			Config akkaConfig = AkkaUtils.getAkkaConfig(configuration, new Some<>(address));
    +			LOG.debug("Using akka configuration\n " + akkaConfig);
    +			taskManagerSystem = AkkaUtils.createActorSystem(akkaConfig);
    +		} catch (Throwable t) {
    +			if (t instanceof org.jboss.netty.channel.ChannelException) {
    +				Throwable cause = t.getCause();
    +				if (cause != null && t.getCause() instanceof java.net.BindException) {
    +					String address = NetUtils.hostAndPortToUrlString(taskManagerHostname, actorSystemPort);
    +					throw new IOException("Unable to bind TaskManager actor system to address " +
    +						address + " - " + cause.getMessage(), t);
    +				}
    +			}
    +			throw new Exception("Could not create TaskManager actor system", t);
    +		}
    +
    +		// start all the TaskManager services (network stack,  library cache, ...)
    +		// and the TaskManager actor
    +		try {
    +			LOG.info("Starting TaskManager actor");
    +			ActorRef taskManagerActor = startTaskManagerComponentsAndActor(
    +				configuration,
    +				resourceID,
    +				taskManagerSystem,
    +				taskManagerHostname,
    +				TASK_MANAGER_NAME,
    +				null,
    +				false,
    +				taskManagerClass);
    +
    +			// start a process reaper that watches the JobManager. If the TaskManager actor dies,
    +			// the process reaper will kill the JVM process (to ensure easy failure detection)
    +			LOG.debug("Starting TaskManager process reaper");
    +
    +			taskManagerSystem.actorOf(
    +				Props.create(ProcessReaper.class, taskManagerActor, LOG, RUNTIME_FAILURE_RETURN_CODE),
    +				"TaskManager_Process_Reaper");
    +
    +			// if desired, start the logging daemon that periodically logs the memory usage information
    +			if (LOG.isInfoEnabled() && configuration.getBoolean(
    +				ConfigConstants.TASK_MANAGER_DEBUG_MEMORY_USAGE_START_LOG_THREAD,
    +				ConfigConstants.DEFAULT_TASK_MANAGER_DEBUG_MEMORY_USAGE_START_LOG_THREAD)) {
    +				LOG.info("Starting periodic memory usage logger");
    +
    +				long interval = configuration.getLong(
    +					ConfigConstants.TASK_MANAGER_DEBUG_MEMORY_USAGE_LOG_INTERVAL_MS,
    +					ConfigConstants.DEFAULT_TASK_MANAGER_DEBUG_MEMORY_USAGE_LOG_INTERVAL_MS);
    +
    +				MemoryLogger logger = new MemoryLogger(LOG, interval, taskManagerSystem);
    +				logger.start();
    +			}
    +
    +			// block until everything is done
    +			taskManagerSystem.awaitTermination();
    +		} catch (Throwable t) {
    +			LOG.error("Error while starting up taskManager", t);
    +			try {
    +				taskManagerSystem.shutdown();
    +			} catch (Throwable tt) {
    +				LOG.warn("Could not cleanly shut down actor system", tt);
    +			}
    +			throw t;
    +		}
    +	}
    +
    +	// --------------------------------------------------------------------------
    +	//  Starting and running the TaskManager
    +	// --------------------------------------------------------------------------
    +
    +	/**
    +	 * @param configuration                 The configuration for the TaskManager.
    +	 * @param resourceID                    The id of the resource which the task manager will run on.
    +	 * @param actorSystem                  The actor system that should run the TaskManager actor.
    +	 * @param taskManagerHostname       The hostname/address that describes the TaskManager's data location.
    +	 * @param taskManagerActorName      Optionally the name of the TaskManager actor. If none is given,
    +	 *                                      the actor will use a random name.
    +	 * @param leaderRetrievalService        Optionally, a leader retrieval service can be provided. If none is given,
    +	 *                                      then a LeaderRetrievalService is constructed from the configuration.
    +	 * @param localTaskManagerCommunication If true, the TaskManager will not initiate the TCP network stack.
    +	 * @param taskManagerClass      The class of the TaskManager actor. May be used to give
    +	 *                                      subclasses that understand additional actor messages.
    +	 * @return An ActorRef to the TaskManager actor.
    +	 * @throws org.apache.flink.configuration.IllegalConfigurationException     Thrown, if the given config contains illegal values.
    +	 * @throws java.io.IOException      Thrown, if any of the I/O components (such as buffer pools,
    +	 *                                       I/O manager, ...) cannot be properly started.
    +	 * @throws java.lang.Exception      Thrown is some other error occurs while parsing the configuration
    +	 *                                      or starting the TaskManager components.
    +	 */
    +	public static ActorRef startTaskManagerComponentsAndActor(
    +		Configuration configuration,
    +		ResourceID resourceID,
    +		ActorSystem actorSystem,
    +		String taskManagerHostname,
    +		String taskManagerActorName,
    +		LeaderRetrievalService leaderRetrievalService,
    +		boolean localTaskManagerCommunication,
    +		Class<? extends TaskManager> taskManagerClass) throws Exception {
    +
    +		Tuple4<TaskManagerConfiguration, NetworkEnvironmentConfiguration, InstanceConnectionInfo, MemoryType> tuple4
    +			= parseTaskManagerConfiguration(configuration, taskManagerHostname, localTaskManagerCommunication);
    +
    +		TaskManagerConfiguration taskManagerConfig = tuple4._1();
    +		NetworkEnvironmentConfiguration netConfig = tuple4._2();
    +		InstanceConnectionInfo connectionInfo = tuple4._3();
    +		MemoryType memType = tuple4._4();
    +
    +		// pre-start checks
    +		checkTempDirs(taskManagerConfig.tmpDirPaths());
    +
    +		ExecutionContext executionContext = ExecutionContexts$.MODULE$.fromExecutor(new ForkJoinPool());
    +
    +		// we start the network first, to make sure it can allocate its buffers first
    +		NetworkEnvironment network = new NetworkEnvironment(executionContext, taskManagerConfig.timeout(), netConfig);
    +
    +		// computing the amount of memory to use depends on how much memory is available
    +		// it strictly needs to happen AFTER the network stack has been initialized
    +
    +		// check if a value has been configured
    +		long configuredMemory = configuration.getLong(ConfigConstants.TASK_MANAGER_MEMORY_SIZE_KEY, -1L);
    +		checkConfigParameter(configuredMemory == -1 || configuredMemory > 0, configuredMemory,
    +			ConfigConstants.TASK_MANAGER_MEMORY_SIZE_KEY,
    +			"MemoryManager needs at least one MB of memory. " +
    +				"If you leave this config parameter empty, the system automatically " +
    +				"pick a fraction of the available memory.");
    +
    +		long memorySize;
    +		boolean preAllocateMemory = configuration.getBoolean(
    +			ConfigConstants.TASK_MANAGER_MEMORY_PRE_ALLOCATE_KEY,
    +			ConfigConstants.DEFAULT_TASK_MANAGER_MEMORY_PRE_ALLOCATE);
    +		if (configuredMemory > 0) {
    +			if (preAllocateMemory) {
    +				LOG.info("Using {} MB for managed memory." , configuredMemory);
    +			} else {
    +				LOG.info("Limiting managed memory to {} MB, memory will be allocated lazily." , configuredMemory);
    +			}
    +			memorySize = configuredMemory << 20; // megabytes to bytes
    +		} else {
    +			float fraction = configuration.getFloat(
    +				ConfigConstants.TASK_MANAGER_MEMORY_FRACTION_KEY,
    +				ConfigConstants.DEFAULT_MEMORY_MANAGER_MEMORY_FRACTION);
    +			checkConfigParameter(fraction > 0.0f && fraction < 1.0f, fraction,
    +				ConfigConstants.TASK_MANAGER_MEMORY_FRACTION_KEY,
    +				"MemoryManager fraction of the free memory must be between 0.0 and 1.0");
    +
    +			if (memType == MemoryType.HEAP) {
    +				long relativeMemSize = (long) (EnvironmentInformation.getSizeOfFreeHeapMemoryWithDefrag() * fraction);
    +				if (preAllocateMemory) {
    +					LOG.info("Using {} of the currently free heap space for managed heap memory ({} MB)." ,
    +						fraction , relativeMemSize >> 20);
    +				} else {
    +					LOG.info("Limiting managed memory to {} of the currently free heap space ({} MB), " +
    +						"memory will be allocated lazily." , fraction , relativeMemSize >> 20);
    +				}
    +				memorySize = relativeMemSize;
    +			} else if (memType == MemoryType.OFF_HEAP) {
    +				// The maximum heap memory has been adjusted according to the fraction
    +				long maxMemory = EnvironmentInformation.getMaxJvmHeapMemory();
    +				long directMemorySize = (long) (maxMemory / (1.0 - fraction) * fraction);
    +				if (preAllocateMemory) {
    +					LOG.info("Using {} of the maximum memory size for managed off-heap memory ({} MB)." ,
    +						fraction, directMemorySize >> 20);
    +				} else {
    +					LOG.info("Limiting managed memory to {} of the maximum memory size ({} MB)," +
    +						" memory will be allocated lazily.", fraction, directMemorySize >> 20);
    +				}
    +				memorySize = directMemorySize;
    +			} else {
    +				throw new RuntimeException("No supported memory type detected.");
    +			}
    +		}
    +
    +		// now start the memory manager
    +		MemoryManager memoryManager;
    +		try {
    +			memoryManager = new MemoryManager(
    +				memorySize,
    +				taskManagerConfig.numberOfSlots(),
    +				netConfig.networkBufferSize(),
    +				memType,
    +				preAllocateMemory);
    +		} catch (OutOfMemoryError e) {
    +			if (memType == MemoryType.HEAP) {
    +				throw new Exception("OutOfMemory error (" + e.getMessage() +
    +					") while allocating the TaskManager heap memory (" + memorySize + " bytes).", e);
    +			} else if (memType == MemoryType.OFF_HEAP) {
    +				throw new Exception("OutOfMemory error (" + e.getMessage() +
    +					") while allocating the TaskManager off-heap memory (" + memorySize +
    +					" bytes).Try increasing the maximum direct memory (-XX:MaxDirectMemorySize)", e);
    +			} else {
    +				throw e;
    +			}
    +		}
    +
    +		// start the I/O manager, it will create some temp directories.
    +		IOManager ioManager = new IOManagerAsync(taskManagerConfig.tmpDirPaths());
    +
    +		if (leaderRetrievalService == null){
    +			leaderRetrievalService = LeaderRetrievalUtils.createLeaderRetrievalService(configuration);
    +		}
    +
    +		// create the actor properties (which define the actor constructor parameters)
    +		Props tmProps = Props.create(
    +			taskManagerClass,
    +			taskManagerConfig,
    +			resourceID,
    +			connectionInfo,
    +			memoryManager,
    +			ioManager,
    +			network,
    +			taskManagerConfig.numberOfSlots(),
    +			leaderRetrievalService);
    +
    +		ActorRef taskManagerActorRef;
    +		if (taskManagerActorName != null && !taskManagerActorName.equals("")) {
    +			taskManagerActorRef = actorSystem.actorOf(tmProps, taskManagerActorName);
    +		} else {
    +			taskManagerActorRef = actorSystem.actorOf(tmProps);
    +		}
    +
    +		return taskManagerActorRef;
    +	}
    +
    +	// --------------------------------------------------------------------------
    +	//  Parsing and checking the TaskManager Configuration
    +	// --------------------------------------------------------------------------
    +
    +	/**
    +	 * Utility method to extract TaskManager config parameters from the configuration and to
    +	 * sanity check them.
    +	 *
    +	 * @param configuration                 The configuration.
    +	 * @param taskManagerHostname           The host name under which the TaskManager communicates.
    +	 * @param localTaskManagerCommunication True, to skip initializing the network stack.
    +	 *                                      Use only in cases where only one task manager runs.
    +	 * @return A tuple (TaskManagerConfiguration, network configuration,
    +	 * InstanceConnectionInfo, JobManager actor Akka URL).
    +	 */
    +	private static Tuple4<TaskManagerConfiguration, NetworkEnvironmentConfiguration, InstanceConnectionInfo, MemoryType>
    +		parseTaskManagerConfiguration(Configuration configuration, String taskManagerHostname, boolean localTaskManagerCommunication)
    +		throws Exception {
    +
    +		// ------- read values from the config and check them ---------
    +		//                      (a lot of them)
    +
    +		// ----> hosts / ports for communication and data exchange
    +
    +		int dataport = configuration.getInteger(ConfigConstants.TASK_MANAGER_DATA_PORT_KEY,
    +			ConfigConstants.DEFAULT_TASK_MANAGER_DATA_PORT);
    +		if (dataport == 0) {
    +			dataport = NetUtils.getAvailablePort();
    +		}
    +		checkConfigParameter(dataport > 0, dataport, ConfigConstants.TASK_MANAGER_DATA_PORT_KEY,
    +			"Leave config parameter empty or use 0 to let the system choose a port automatically.");
    +
    +		InetAddress taskManagerAddress = InetAddress.getByName(taskManagerHostname);
    +		InstanceConnectionInfo connectionInfo = new InstanceConnectionInfo(taskManagerAddress, dataport);
    +
    +		// ----> memory / network stack (shuffles/broadcasts), task slots, temp directories
    +
    +		// we need this because many configs have been written with a "-1" entry
    +		int slots = configuration.getInteger(ConfigConstants.TASK_MANAGER_NUM_TASK_SLOTS, 1);
    +		if (slots == -1) {
    +			slots = 1;
    +		}
    +		checkConfigParameter(slots >= 1, slots, ConfigConstants.TASK_MANAGER_NUM_TASK_SLOTS,
    +			"Number of task slots must be at least one.");
    +
    +		int numNetworkBuffers = configuration.getInteger(
    +			ConfigConstants.TASK_MANAGER_NETWORK_NUM_BUFFERS_KEY,
    +			ConfigConstants.DEFAULT_TASK_MANAGER_NETWORK_NUM_BUFFERS);
    +		checkConfigParameter(numNetworkBuffers > 0, numNetworkBuffers,
    +			ConfigConstants.TASK_MANAGER_NETWORK_NUM_BUFFERS_KEY, "");
    +
    +		int pageSize = configuration.getInteger(
    +			ConfigConstants.TASK_MANAGER_MEMORY_SEGMENT_SIZE_KEY,
    +			ConfigConstants.DEFAULT_TASK_MANAGER_MEMORY_SEGMENT_SIZE);
    +		// check page size of for minimum size
    +		checkConfigParameter(pageSize >= MemoryManager.MIN_PAGE_SIZE, pageSize,
    +			ConfigConstants.TASK_MANAGER_MEMORY_SEGMENT_SIZE_KEY,
    +			"Minimum memory segment size is " + MemoryManager.MIN_PAGE_SIZE);
    +		// check page size for power of two
    +		checkConfigParameter(MathUtils.isPowerOf2(pageSize), pageSize,
    +			ConfigConstants.TASK_MANAGER_MEMORY_SEGMENT_SIZE_KEY,
    +			"Memory segment size must be a power of 2.");
    +
    +		// check whether we use heap or off-heap memory
    +		MemoryType memType;
    +		if (configuration.getBoolean(ConfigConstants.TASK_MANAGER_MEMORY_OFF_HEAP_KEY, false)) {
    +			memType = MemoryType.OFF_HEAP;
    +		} else {
    +			memType = MemoryType.HEAP;
    +		}
    +
    +		// initialize the memory segment factory accordingly
    +		if (memType == MemoryType.HEAP) {
    +			if (!MemorySegmentFactory.isInitialized()) {
    +				MemorySegmentFactory.initializeFactory(HeapMemorySegment.FACTORY);
    +			} else if (MemorySegmentFactory.getFactory() != HeapMemorySegment.FACTORY) {
    +				throw new Exception("Memory type is set to heap memory, but memory segment " +
    +					"factory has been initialized for off-heap memory segments");
    +			}
    +		} else {
    +			if (!MemorySegmentFactory.isInitialized()) {
    +				MemorySegmentFactory.initializeFactory(HybridMemorySegment.FACTORY);
    +			} else if (MemorySegmentFactory.getFactory() != HybridMemorySegment.FACTORY) {
    +				throw new Exception("Memory type is set to off-heap memory, but memory segment " +
    +					"factory has been initialized for heap memory segments");
    +			}
    +		}
    +
    +		String[] tmpDirs = configuration.getString(
    +			ConfigConstants.TASK_MANAGER_TMP_DIR_KEY,
    +			ConfigConstants.DEFAULT_TASK_MANAGER_TMP_PATH).split(",|" + File.pathSeparator);
    +
    +		NettyConfig nettyConfig = null;
    +		if (!localTaskManagerCommunication) {
    +			nettyConfig = new NettyConfig(connectionInfo.address(), connectionInfo.dataPort(), pageSize, slots, configuration);
    +		}
    +
    +		// Default spill I/O mode for intermediate results
    +		String syncOrAsync = configuration.getString(
    +			ConfigConstants.TASK_MANAGER_NETWORK_DEFAULT_IO_MODE,
    +			ConfigConstants.DEFAULT_TASK_MANAGER_NETWORK_DEFAULT_IO_MODE);
    +
    +		IOMode ioMode;
    +		if (syncOrAsync == "async") {
    +			ioMode = IOManager.IOMode.ASYNC;
    +		} else {
    +			ioMode = IOManager.IOMode.SYNC;
    +		}
    +
    +		NetworkEnvironmentConfiguration networkConfig = new NetworkEnvironmentConfiguration(
    +			numNetworkBuffers,
    +			pageSize,
    +			memType,
    +			ioMode,
    +			nettyConfig == null ? Option.<NettyConfig>empty() : new Some<>(nettyConfig),
    +			new Tuple2<>(500, 3000));
    +
    +		// ----> timeouts, library caching, profiling
    +
    +		FiniteDuration timeout;
    +		try {
    +			timeout = AkkaUtils.getTimeout(configuration);
    +		} catch (Exception e) {
    +			throw new IllegalArgumentException(
    +				"Invalid format for '" + ConfigConstants.AKKA_ASK_TIMEOUT +
    +					"'.Use formats like '50 s' or '1 min' to specify the timeout.");
    +		}
    +		LOG.info("Messages between TaskManager and JobManager have a max timeout of " + timeout);
    +
    +		long cleanupInterval = configuration.getLong(
    +			ConfigConstants.LIBRARY_CACHE_MANAGER_CLEANUP_INTERVAL,
    +			ConfigConstants.DEFAULT_LIBRARY_CACHE_MANAGER_CLEANUP_INTERVAL) * 1000;
    +
    +		FiniteDuration finiteRegistrationDuration = null;
    +		try {
    +			Duration maxRegistrationDuration = Duration.create(configuration.getString(
    +				ConfigConstants.TASK_MANAGER_MAX_REGISTRATION_DURATION,
    +				ConfigConstants.DEFAULT_TASK_MANAGER_MAX_REGISTRATION_DURATION));
    +			if (maxRegistrationDuration.isFinite()) {
    +				finiteRegistrationDuration = new FiniteDuration(maxRegistrationDuration.toSeconds(), TimeUnit.SECONDS);
    +			}
    +		} catch (NumberFormatException e) {
    +			throw new IllegalArgumentException("Invalid format for parameter " +
    +				ConfigConstants.TASK_MANAGER_MAX_REGISTRATION_DURATION, e);
    +		}
    +
    +		FiniteDuration initialRegistrationPause;
    +		try {
    +			Duration pause = Duration.create(configuration.getString(
    +				ConfigConstants.TASK_MANAGER_INITIAL_REGISTRATION_PAUSE,
    +				ConfigConstants.DEFAULT_TASK_MANAGER_INITIAL_REGISTRATION_PAUSE));
    +			if (pause.isFinite()) {
    +				initialRegistrationPause = new FiniteDuration(pause.toSeconds(), TimeUnit.SECONDS);
    +			} else {
    +				throw new IllegalArgumentException("The initial registration pause must be finite: " + pause);
    +			}
    +		} catch (NumberFormatException e) {
    +			throw new IllegalArgumentException("Invalid format for parameter " +
    +				ConfigConstants.TASK_MANAGER_INITIAL_REGISTRATION_PAUSE, e);
    +		}
    +
    +		FiniteDuration maxRegistrationPause;
    +		try {
    +			Duration pause = Duration.create(configuration.getString(
    +				ConfigConstants.TASK_MANAGER_MAX_REGISTARTION_PAUSE,
    +				ConfigConstants.DEFAULT_TASK_MANAGER_MAX_REGISTRATION_PAUSE));
    +			if (pause.isFinite()) {
    +				maxRegistrationPause = new FiniteDuration(pause.toSeconds(), TimeUnit.SECONDS);
    +			} else {
    +				throw new IllegalArgumentException("The maximum registration pause must be finite: " + pause);
    +			}
    +		} catch (NumberFormatException e) {
    +			throw new IllegalArgumentException("Invalid format for parameter " +
    +				ConfigConstants.TASK_MANAGER_INITIAL_REGISTRATION_PAUSE, e);
    +		}
    +
    +		FiniteDuration refusedRegistrationPause;
    +		try {
    +			Duration pause = Duration.create(configuration.getString(
    +				ConfigConstants.TASK_MANAGER_REFUSED_REGISTRATION_PAUSE,
    +				ConfigConstants.DEFAULT_TASK_MANAGER_REFUSED_REGISTRATION_PAUSE));
    +			if (pause.isFinite()) {
    +				refusedRegistrationPause = new FiniteDuration(pause.toSeconds(), TimeUnit.SECONDS);
    +			} else {
    +				throw new IllegalArgumentException("The refused registration pause must be finite: " + pause);
    +			}
    +		} catch (NumberFormatException e) {
    +			throw new IllegalArgumentException("Invalid format for parameter " +
    +				ConfigConstants.TASK_MANAGER_INITIAL_REGISTRATION_PAUSE, e);
    +		}
    +
    +		TaskManagerConfiguration taskManagerConfig = new TaskManagerConfiguration(
    --- End diff --
    
    If add the new NetworkEnvironmentConfiguration class in java, it will cause problems when generate NetworkEnvironment. So i did not do that currently.


---
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.
---

[GitHub] flink pull request #2400: [FLINK-4363] Implement TaskManager basic startup o...

Posted by mxm <gi...@git.apache.org>.
Github user mxm commented on a diff in the pull request:

    https://github.com/apache/flink/pull/2400#discussion_r75647899
  
    --- Diff: flink-runtime/src/main/java/org/apache/flink/runtime/rpc/taskexecutor/TaskExecutor.java ---
    @@ -35,27 +79,634 @@
      */
     public class TaskExecutor extends RpcEndpoint<TaskExecutorGateway> {
     
    -	/** The unique resource ID of this TaskExecutor */
    +	private static final Logger LOG = LoggerFactory.getLogger(TaskExecutor.class);
    +
    +	/** Return code for critical errors during the runtime */
    +	private static final int RUNTIME_FAILURE_RETURN_CODE = 2;
    +
    +	/** The name of the TaskManager actor */
    +	private static final String TASK_MANAGER_NAME = "taskmanager";
    +
    +	/** The unique resource ID of this TaskManager */
     	private final ResourceID resourceID;
     
     	/** The access to the leader election and metadata storage services */
     	private final HighAvailabilityServices haServices;
     
    -	// --------- resource manager --------
    +	/** The task manager configuration */
    +	private final TaskManagerConfiguration taskManagerConfig;
    +
    +	/** The connection information of the task manager */
    +	private final InstanceConnectionInfo connectionInfo;
    +
    +	/** The I/O manager component in the task manager */
    +	private final IOManager ioManager;
    +
    +	/** The memory manager component in the task manager */
    +	private final MemoryManager memoryManager;
    +
    +	/** The network component in the task manager */
    +	private final NetworkEnvironment networkEnvironment;
    +
    +	/** The number of slots in the task manager, should be 1 for YARN */
    +	private final int numberOfSlots;
     
    +	// --------- resource manager --------
     	private TaskExecutorToResourceManagerConnection resourceManagerConnection;
     
     	// ------------------------------------------------------------------------
     
     	public TaskExecutor(
    +			TaskManagerConfiguration taskManagerConfig,
    +			ResourceID resourceID,
    +			InstanceConnectionInfo connectionInfo,
    +			MemoryManager memoryManager,
    +			IOManager ioManager,
    +			NetworkEnvironment networkEnvironment,
    +			int numberOfSlots,
     			RpcService rpcService,
    -			HighAvailabilityServices haServices,
    -			ResourceID resourceID) {
    +			HighAvailabilityServices haServices) {
     
     		super(rpcService);
     
    -		this.haServices = checkNotNull(haServices);
    +		this.taskManagerConfig = checkNotNull(taskManagerConfig);
     		this.resourceID = checkNotNull(resourceID);
    +		this.connectionInfo = checkNotNull(connectionInfo);
    +		this.memoryManager = checkNotNull(memoryManager);
    +		this.ioManager = checkNotNull(ioManager);
    +		this.networkEnvironment = checkNotNull(networkEnvironment);
    +		this.numberOfSlots = checkNotNull(numberOfSlots);
    +		this.haServices = checkNotNull(haServices);
    +	}
    +
    +	/**
    +	 * Starts and runs the TaskManager.
    +	 * <p/>
    +	 * This method first tries to select the network interface to use for the TaskManager
    +	 * communication. The network interface is used both for the actor communication
    +	 * (coordination) as well as for the data exchange between task managers. Unless
    +	 * the hostname/interface is explicitly configured in the configuration, this
    +	 * method will try out various interfaces and methods to connect to the JobManager
    +	 * and select the one where the connection attempt is successful.
    +	 * <p/>
    +	 * After selecting the network interface, this method brings up an actor system
    +	 * for the TaskManager and its actors, starts the TaskManager's services
    +	 * (library cache, shuffle network stack, ...), and starts the TaskManager itself.
    +	 *
    +	 * @param configuration    The configuration for the TaskManager.
    +	 * @param taskManagerClass The actor class to instantiate.
    +	 *                         Allows to use TaskManager subclasses for example for YARN.
    +	 */
    +	public static void selectNetworkInterfaceAndRunTaskManager(
    +		Configuration configuration,
    +		ResourceID resourceID,
    +		Class<? extends TaskManager> taskManagerClass) throws Exception {
    +
    +		Tuple2<String, Integer> tuple2 = selectNetworkInterfaceAndPort(configuration);
    +
    +		runTaskManager(tuple2._1(), resourceID, tuple2._2(), configuration, taskManagerClass);
    +	}
    +
    +	private static Tuple2<String, Integer> selectNetworkInterfaceAndPort(Configuration configuration)
    +		throws Exception {
    +		String taskManagerHostname = configuration.getString(ConfigConstants.TASK_MANAGER_HOSTNAME_KEY, null);
    +		if (taskManagerHostname != null) {
    +			LOG.info("Using configured hostname/address for TaskManager: " + taskManagerHostname);
    +		} else {
    +			LeaderRetrievalService leaderRetrievalService = LeaderRetrievalUtils.createLeaderRetrievalService(configuration);
    +			FiniteDuration lookupTimeout = AkkaUtils.getLookupTimeout(configuration);
    +
    +			InetAddress taskManagerAddress = LeaderRetrievalUtils.findConnectingAddress(leaderRetrievalService, lookupTimeout);
    +			taskManagerHostname = taskManagerAddress.getHostName();
    +			LOG.info("TaskManager will use hostname/address '{}' ({}) for communication.",
    +				taskManagerHostname, taskManagerAddress.getHostAddress());
    +		}
    +
    +		// if no task manager port has been configured, use 0 (system will pick any free port)
    +		int actorSystemPort = configuration.getInteger(ConfigConstants.TASK_MANAGER_IPC_PORT_KEY, 0);
    +		if (actorSystemPort < 0 || actorSystemPort > 65535) {
    +			throw new IllegalConfigurationException("Invalid value for '" +
    +				ConfigConstants.TASK_MANAGER_IPC_PORT_KEY +
    +				"' (port for the TaskManager actor system) : " + actorSystemPort +
    +				" - Leave config parameter empty or use 0 to let the system choose a port automatically.");
    +		}
    +
    +		return new Tuple2<>(taskManagerHostname, actorSystemPort);
    +	}
    +
    +	/**
    +	 * Starts and runs the TaskManager. Brings up an actor system for the TaskManager and its
    +	 * actors, starts the TaskManager's services (library cache, shuffle network stack, ...),
    +	 * and starts the TaskManager itself.
    +	 * <p/>
    +	 * This method will also spawn a process reaper for the TaskManager (kill the process if
    +	 * the actor fails) and optionally start the JVM memory logging thread.
    +	 *
    +	 * @param taskManagerHostname The hostname/address of the interface where the actor system
    +	 *                            will communicate.
    +	 * @param resourceID          The id of the resource which the task manager will run on.
    +	 * @param actorSystemPort     The port at which the actor system will communicate.
    +	 * @param configuration       The configuration for the TaskManager.
    +	 * @param taskManagerClass    The actor class to instantiate. Allows the use of TaskManager
    +	 *                            subclasses for example for YARN.
    +	 */
    +	private static void runTaskManager(
    +		String taskManagerHostname,
    +		ResourceID resourceID,
    +		int actorSystemPort,
    +		Configuration configuration,
    +		Class<? extends TaskManager> taskManagerClass) throws Exception {
    +
    +		LOG.info("Starting TaskManager");
    +
    +		// Bring up the TaskManager actor system first, bind it to the given address.
    +
    +		LOG.info("Starting TaskManager actor system at " +
    +			NetUtils.hostAndPortToUrlString(taskManagerHostname, actorSystemPort));
    +
    +		ActorSystem taskManagerSystem;
    +		try {
    +			Tuple2<String, Object> address = new Tuple2<String, Object>(taskManagerHostname, actorSystemPort);
    +			Config akkaConfig = AkkaUtils.getAkkaConfig(configuration, new Some<>(address));
    +			LOG.debug("Using akka configuration\n " + akkaConfig);
    +			taskManagerSystem = AkkaUtils.createActorSystem(akkaConfig);
    +		} catch (Throwable t) {
    +			if (t instanceof org.jboss.netty.channel.ChannelException) {
    +				Throwable cause = t.getCause();
    +				if (cause != null && t.getCause() instanceof java.net.BindException) {
    +					String address = NetUtils.hostAndPortToUrlString(taskManagerHostname, actorSystemPort);
    +					throw new IOException("Unable to bind TaskManager actor system to address " +
    +						address + " - " + cause.getMessage(), t);
    +				}
    +			}
    +			throw new Exception("Could not create TaskManager actor system", t);
    +		}
    +
    +		// start all the TaskManager services (network stack,  library cache, ...)
    +		// and the TaskManager actor
    +		try {
    +			LOG.info("Starting TaskManager actor");
    +			ActorRef taskManagerActor = startTaskManagerComponentsAndActor(
    +				configuration,
    +				resourceID,
    +				taskManagerSystem,
    +				taskManagerHostname,
    +				TASK_MANAGER_NAME,
    +				null,
    +				false,
    +				taskManagerClass);
    +
    +			// start a process reaper that watches the JobManager. If the TaskManager actor dies,
    +			// the process reaper will kill the JVM process (to ensure easy failure detection)
    +			LOG.debug("Starting TaskManager process reaper");
    +
    +			taskManagerSystem.actorOf(
    +				Props.create(ProcessReaper.class, taskManagerActor, LOG, RUNTIME_FAILURE_RETURN_CODE),
    +				"TaskManager_Process_Reaper");
    +
    +			// if desired, start the logging daemon that periodically logs the memory usage information
    +			if (LOG.isInfoEnabled() && configuration.getBoolean(
    +				ConfigConstants.TASK_MANAGER_DEBUG_MEMORY_USAGE_START_LOG_THREAD,
    +				ConfigConstants.DEFAULT_TASK_MANAGER_DEBUG_MEMORY_USAGE_START_LOG_THREAD)) {
    +				LOG.info("Starting periodic memory usage logger");
    +
    +				long interval = configuration.getLong(
    +					ConfigConstants.TASK_MANAGER_DEBUG_MEMORY_USAGE_LOG_INTERVAL_MS,
    +					ConfigConstants.DEFAULT_TASK_MANAGER_DEBUG_MEMORY_USAGE_LOG_INTERVAL_MS);
    +
    +				MemoryLogger logger = new MemoryLogger(LOG, interval, taskManagerSystem);
    +				logger.start();
    +			}
    +
    +			// block until everything is done
    +			taskManagerSystem.awaitTermination();
    +		} catch (Throwable t) {
    +			LOG.error("Error while starting up taskManager", t);
    +			try {
    +				taskManagerSystem.shutdown();
    +			} catch (Throwable tt) {
    +				LOG.warn("Could not cleanly shut down actor system", tt);
    +			}
    +			throw t;
    +		}
    +	}
    +
    +	// --------------------------------------------------------------------------
    +	//  Starting and running the TaskManager
    +	// --------------------------------------------------------------------------
    +
    +	/**
    +	 * @param configuration                 The configuration for the TaskManager.
    +	 * @param resourceID                    The id of the resource which the task manager will run on.
    +	 * @param actorSystem                  The actor system that should run the TaskManager actor.
    +	 * @param taskManagerHostname       The hostname/address that describes the TaskManager's data location.
    +	 * @param taskManagerActorName      Optionally the name of the TaskManager actor. If none is given,
    +	 *                                      the actor will use a random name.
    +	 * @param leaderRetrievalService        Optionally, a leader retrieval service can be provided. If none is given,
    +	 *                                      then a LeaderRetrievalService is constructed from the configuration.
    +	 * @param localTaskManagerCommunication If true, the TaskManager will not initiate the TCP network stack.
    +	 * @param taskManagerClass      The class of the TaskManager actor. May be used to give
    +	 *                                      subclasses that understand additional actor messages.
    +	 * @return An ActorRef to the TaskManager actor.
    +	 * @throws org.apache.flink.configuration.IllegalConfigurationException     Thrown, if the given config contains illegal values.
    +	 * @throws java.io.IOException      Thrown, if any of the I/O components (such as buffer pools,
    +	 *                                       I/O manager, ...) cannot be properly started.
    +	 * @throws java.lang.Exception      Thrown is some other error occurs while parsing the configuration
    +	 *                                      or starting the TaskManager components.
    +	 */
    +	public static ActorRef startTaskManagerComponentsAndActor(
    +		Configuration configuration,
    +		ResourceID resourceID,
    +		ActorSystem actorSystem,
    +		String taskManagerHostname,
    +		String taskManagerActorName,
    +		LeaderRetrievalService leaderRetrievalService,
    +		boolean localTaskManagerCommunication,
    +		Class<? extends TaskManager> taskManagerClass) throws Exception {
    +
    +		Tuple4<TaskManagerConfiguration, NetworkEnvironmentConfiguration, InstanceConnectionInfo, MemoryType> tuple4
    +			= parseTaskManagerConfiguration(configuration, taskManagerHostname, localTaskManagerCommunication);
    +
    +		TaskManagerConfiguration taskManagerConfig = tuple4._1();
    +		NetworkEnvironmentConfiguration netConfig = tuple4._2();
    +		InstanceConnectionInfo connectionInfo = tuple4._3();
    +		MemoryType memType = tuple4._4();
    +
    +		// pre-start checks
    +		checkTempDirs(taskManagerConfig.tmpDirPaths());
    +
    +		ExecutionContext executionContext = ExecutionContexts$.MODULE$.fromExecutor(new ForkJoinPool());
    +
    +		// we start the network first, to make sure it can allocate its buffers first
    +		NetworkEnvironment network = new NetworkEnvironment(executionContext, taskManagerConfig.timeout(), netConfig);
    +
    +		// computing the amount of memory to use depends on how much memory is available
    +		// it strictly needs to happen AFTER the network stack has been initialized
    +
    +		// check if a value has been configured
    +		long configuredMemory = configuration.getLong(ConfigConstants.TASK_MANAGER_MEMORY_SIZE_KEY, -1L);
    +		checkConfigParameter(configuredMemory == -1 || configuredMemory > 0, configuredMemory,
    +			ConfigConstants.TASK_MANAGER_MEMORY_SIZE_KEY,
    +			"MemoryManager needs at least one MB of memory. " +
    +				"If you leave this config parameter empty, the system automatically " +
    +				"pick a fraction of the available memory.");
    +
    +		long memorySize;
    +		boolean preAllocateMemory = configuration.getBoolean(
    +			ConfigConstants.TASK_MANAGER_MEMORY_PRE_ALLOCATE_KEY,
    +			ConfigConstants.DEFAULT_TASK_MANAGER_MEMORY_PRE_ALLOCATE);
    +		if (configuredMemory > 0) {
    +			if (preAllocateMemory) {
    +				LOG.info("Using {} MB for managed memory." , configuredMemory);
    +			} else {
    +				LOG.info("Limiting managed memory to {} MB, memory will be allocated lazily." , configuredMemory);
    +			}
    +			memorySize = configuredMemory << 20; // megabytes to bytes
    +		} else {
    +			float fraction = configuration.getFloat(
    +				ConfigConstants.TASK_MANAGER_MEMORY_FRACTION_KEY,
    +				ConfigConstants.DEFAULT_MEMORY_MANAGER_MEMORY_FRACTION);
    +			checkConfigParameter(fraction > 0.0f && fraction < 1.0f, fraction,
    +				ConfigConstants.TASK_MANAGER_MEMORY_FRACTION_KEY,
    +				"MemoryManager fraction of the free memory must be between 0.0 and 1.0");
    +
    +			if (memType == MemoryType.HEAP) {
    +				long relativeMemSize = (long) (EnvironmentInformation.getSizeOfFreeHeapMemoryWithDefrag() * fraction);
    +				if (preAllocateMemory) {
    +					LOG.info("Using {} of the currently free heap space for managed heap memory ({} MB)." ,
    +						fraction , relativeMemSize >> 20);
    +				} else {
    +					LOG.info("Limiting managed memory to {} of the currently free heap space ({} MB), " +
    +						"memory will be allocated lazily." , fraction , relativeMemSize >> 20);
    +				}
    +				memorySize = relativeMemSize;
    +			} else if (memType == MemoryType.OFF_HEAP) {
    +				// The maximum heap memory has been adjusted according to the fraction
    +				long maxMemory = EnvironmentInformation.getMaxJvmHeapMemory();
    +				long directMemorySize = (long) (maxMemory / (1.0 - fraction) * fraction);
    +				if (preAllocateMemory) {
    +					LOG.info("Using {} of the maximum memory size for managed off-heap memory ({} MB)." ,
    +						fraction, directMemorySize >> 20);
    +				} else {
    +					LOG.info("Limiting managed memory to {} of the maximum memory size ({} MB)," +
    +						" memory will be allocated lazily.", fraction, directMemorySize >> 20);
    +				}
    +				memorySize = directMemorySize;
    +			} else {
    +				throw new RuntimeException("No supported memory type detected.");
    +			}
    +		}
    +
    +		// now start the memory manager
    +		MemoryManager memoryManager;
    +		try {
    +			memoryManager = new MemoryManager(
    +				memorySize,
    +				taskManagerConfig.numberOfSlots(),
    +				netConfig.networkBufferSize(),
    +				memType,
    +				preAllocateMemory);
    +		} catch (OutOfMemoryError e) {
    +			if (memType == MemoryType.HEAP) {
    +				throw new Exception("OutOfMemory error (" + e.getMessage() +
    +					") while allocating the TaskManager heap memory (" + memorySize + " bytes).", e);
    +			} else if (memType == MemoryType.OFF_HEAP) {
    +				throw new Exception("OutOfMemory error (" + e.getMessage() +
    +					") while allocating the TaskManager off-heap memory (" + memorySize +
    +					" bytes).Try increasing the maximum direct memory (-XX:MaxDirectMemorySize)", e);
    +			} else {
    +				throw e;
    +			}
    +		}
    +
    +		// start the I/O manager, it will create some temp directories.
    +		IOManager ioManager = new IOManagerAsync(taskManagerConfig.tmpDirPaths());
    +
    +		if (leaderRetrievalService == null){
    +			leaderRetrievalService = LeaderRetrievalUtils.createLeaderRetrievalService(configuration);
    +		}
    +
    +		// create the actor properties (which define the actor constructor parameters)
    +		Props tmProps = Props.create(
    +			taskManagerClass,
    +			taskManagerConfig,
    +			resourceID,
    +			connectionInfo,
    +			memoryManager,
    +			ioManager,
    +			network,
    +			taskManagerConfig.numberOfSlots(),
    +			leaderRetrievalService);
    +
    +		ActorRef taskManagerActorRef;
    +		if (taskManagerActorName != null && !taskManagerActorName.equals("")) {
    +			taskManagerActorRef = actorSystem.actorOf(tmProps, taskManagerActorName);
    +		} else {
    +			taskManagerActorRef = actorSystem.actorOf(tmProps);
    +		}
    +
    +		return taskManagerActorRef;
    +	}
    +
    +	// --------------------------------------------------------------------------
    +	//  Parsing and checking the TaskManager Configuration
    +	// --------------------------------------------------------------------------
    +
    +	/**
    +	 * Utility method to extract TaskManager config parameters from the configuration and to
    +	 * sanity check them.
    +	 *
    +	 * @param configuration                 The configuration.
    +	 * @param taskManagerHostname           The host name under which the TaskManager communicates.
    +	 * @param localTaskManagerCommunication True, to skip initializing the network stack.
    +	 *                                      Use only in cases where only one task manager runs.
    +	 * @return A tuple (TaskManagerConfiguration, network configuration,
    +	 * InstanceConnectionInfo, JobManager actor Akka URL).
    +	 */
    +	private static Tuple4<TaskManagerConfiguration, NetworkEnvironmentConfiguration, InstanceConnectionInfo, MemoryType>
    +		parseTaskManagerConfiguration(Configuration configuration, String taskManagerHostname, boolean localTaskManagerCommunication)
    +		throws Exception {
    +
    +		// ------- read values from the config and check them ---------
    +		//                      (a lot of them)
    +
    +		// ----> hosts / ports for communication and data exchange
    +
    +		int dataport = configuration.getInteger(ConfigConstants.TASK_MANAGER_DATA_PORT_KEY,
    +			ConfigConstants.DEFAULT_TASK_MANAGER_DATA_PORT);
    +		if (dataport == 0) {
    +			dataport = NetUtils.getAvailablePort();
    +		}
    +		checkConfigParameter(dataport > 0, dataport, ConfigConstants.TASK_MANAGER_DATA_PORT_KEY,
    +			"Leave config parameter empty or use 0 to let the system choose a port automatically.");
    +
    +		InetAddress taskManagerAddress = InetAddress.getByName(taskManagerHostname);
    +		InstanceConnectionInfo connectionInfo = new InstanceConnectionInfo(taskManagerAddress, dataport);
    +
    +		// ----> memory / network stack (shuffles/broadcasts), task slots, temp directories
    +
    +		// we need this because many configs have been written with a "-1" entry
    +		int slots = configuration.getInteger(ConfigConstants.TASK_MANAGER_NUM_TASK_SLOTS, 1);
    +		if (slots == -1) {
    +			slots = 1;
    +		}
    +		checkConfigParameter(slots >= 1, slots, ConfigConstants.TASK_MANAGER_NUM_TASK_SLOTS,
    +			"Number of task slots must be at least one.");
    +
    +		int numNetworkBuffers = configuration.getInteger(
    +			ConfigConstants.TASK_MANAGER_NETWORK_NUM_BUFFERS_KEY,
    +			ConfigConstants.DEFAULT_TASK_MANAGER_NETWORK_NUM_BUFFERS);
    +		checkConfigParameter(numNetworkBuffers > 0, numNetworkBuffers,
    +			ConfigConstants.TASK_MANAGER_NETWORK_NUM_BUFFERS_KEY, "");
    +
    +		int pageSize = configuration.getInteger(
    +			ConfigConstants.TASK_MANAGER_MEMORY_SEGMENT_SIZE_KEY,
    +			ConfigConstants.DEFAULT_TASK_MANAGER_MEMORY_SEGMENT_SIZE);
    +		// check page size of for minimum size
    +		checkConfigParameter(pageSize >= MemoryManager.MIN_PAGE_SIZE, pageSize,
    +			ConfigConstants.TASK_MANAGER_MEMORY_SEGMENT_SIZE_KEY,
    +			"Minimum memory segment size is " + MemoryManager.MIN_PAGE_SIZE);
    +		// check page size for power of two
    +		checkConfigParameter(MathUtils.isPowerOf2(pageSize), pageSize,
    +			ConfigConstants.TASK_MANAGER_MEMORY_SEGMENT_SIZE_KEY,
    +			"Memory segment size must be a power of 2.");
    +
    +		// check whether we use heap or off-heap memory
    +		MemoryType memType;
    +		if (configuration.getBoolean(ConfigConstants.TASK_MANAGER_MEMORY_OFF_HEAP_KEY, false)) {
    +			memType = MemoryType.OFF_HEAP;
    +		} else {
    +			memType = MemoryType.HEAP;
    +		}
    +
    +		// initialize the memory segment factory accordingly
    +		if (memType == MemoryType.HEAP) {
    +			if (!MemorySegmentFactory.isInitialized()) {
    +				MemorySegmentFactory.initializeFactory(HeapMemorySegment.FACTORY);
    +			} else if (MemorySegmentFactory.getFactory() != HeapMemorySegment.FACTORY) {
    +				throw new Exception("Memory type is set to heap memory, but memory segment " +
    +					"factory has been initialized for off-heap memory segments");
    +			}
    +		} else {
    +			if (!MemorySegmentFactory.isInitialized()) {
    +				MemorySegmentFactory.initializeFactory(HybridMemorySegment.FACTORY);
    +			} else if (MemorySegmentFactory.getFactory() != HybridMemorySegment.FACTORY) {
    +				throw new Exception("Memory type is set to off-heap memory, but memory segment " +
    +					"factory has been initialized for heap memory segments");
    +			}
    +		}
    +
    +		String[] tmpDirs = configuration.getString(
    +			ConfigConstants.TASK_MANAGER_TMP_DIR_KEY,
    +			ConfigConstants.DEFAULT_TASK_MANAGER_TMP_PATH).split(",|" + File.pathSeparator);
    +
    +		NettyConfig nettyConfig = null;
    +		if (!localTaskManagerCommunication) {
    +			nettyConfig = new NettyConfig(connectionInfo.address(), connectionInfo.dataPort(), pageSize, slots, configuration);
    +		}
    +
    +		// Default spill I/O mode for intermediate results
    +		String syncOrAsync = configuration.getString(
    +			ConfigConstants.TASK_MANAGER_NETWORK_DEFAULT_IO_MODE,
    +			ConfigConstants.DEFAULT_TASK_MANAGER_NETWORK_DEFAULT_IO_MODE);
    +
    +		IOMode ioMode;
    +		if (syncOrAsync == "async") {
    +			ioMode = IOManager.IOMode.ASYNC;
    +		} else {
    +			ioMode = IOManager.IOMode.SYNC;
    +		}
    +
    +		NetworkEnvironmentConfiguration networkConfig = new NetworkEnvironmentConfiguration(
    +			numNetworkBuffers,
    +			pageSize,
    +			memType,
    +			ioMode,
    +			nettyConfig == null ? Option.<NettyConfig>empty() : new Some<>(nettyConfig),
    +			new Tuple2<>(500, 3000));
    +
    +		// ----> timeouts, library caching, profiling
    +
    +		FiniteDuration timeout;
    +		try {
    +			timeout = AkkaUtils.getTimeout(configuration);
    +		} catch (Exception e) {
    +			throw new IllegalArgumentException(
    +				"Invalid format for '" + ConfigConstants.AKKA_ASK_TIMEOUT +
    +					"'.Use formats like '50 s' or '1 min' to specify the timeout.");
    +		}
    +		LOG.info("Messages between TaskManager and JobManager have a max timeout of " + timeout);
    +
    +		long cleanupInterval = configuration.getLong(
    +			ConfigConstants.LIBRARY_CACHE_MANAGER_CLEANUP_INTERVAL,
    +			ConfigConstants.DEFAULT_LIBRARY_CACHE_MANAGER_CLEANUP_INTERVAL) * 1000;
    +
    +		FiniteDuration finiteRegistrationDuration = null;
    --- End diff --
    
    Would use final without `null` assignment.


---
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.
---

[GitHub] flink pull request #2400: [FLINK-4363] Implement TaskManager basic startup o...

Posted by wangzhijiang999 <gi...@git.apache.org>.
Github user wangzhijiang999 commented on a diff in the pull request:

    https://github.com/apache/flink/pull/2400#discussion_r76367829
  
    --- Diff: flink-runtime/src/main/java/org/apache/flink/runtime/rpc/taskexecutor/TaskExecutor.java ---
    @@ -35,27 +79,634 @@
      */
     public class TaskExecutor extends RpcEndpoint<TaskExecutorGateway> {
     
    -	/** The unique resource ID of this TaskExecutor */
    +	private static final Logger LOG = LoggerFactory.getLogger(TaskExecutor.class);
    +
    +	/** Return code for critical errors during the runtime */
    +	private static final int RUNTIME_FAILURE_RETURN_CODE = 2;
    +
    +	/** The name of the TaskManager actor */
    +	private static final String TASK_MANAGER_NAME = "taskmanager";
    +
    +	/** The unique resource ID of this TaskManager */
     	private final ResourceID resourceID;
     
     	/** The access to the leader election and metadata storage services */
     	private final HighAvailabilityServices haServices;
     
    -	// --------- resource manager --------
    +	/** The task manager configuration */
    +	private final TaskManagerConfiguration taskManagerConfig;
    +
    +	/** The connection information of the task manager */
    +	private final InstanceConnectionInfo connectionInfo;
    +
    +	/** The I/O manager component in the task manager */
    +	private final IOManager ioManager;
    +
    +	/** The memory manager component in the task manager */
    +	private final MemoryManager memoryManager;
    +
    +	/** The network component in the task manager */
    +	private final NetworkEnvironment networkEnvironment;
    +
    +	/** The number of slots in the task manager, should be 1 for YARN */
    +	private final int numberOfSlots;
     
    +	// --------- resource manager --------
     	private TaskExecutorToResourceManagerConnection resourceManagerConnection;
     
     	// ------------------------------------------------------------------------
     
     	public TaskExecutor(
    +			TaskManagerConfiguration taskManagerConfig,
    +			ResourceID resourceID,
    +			InstanceConnectionInfo connectionInfo,
    +			MemoryManager memoryManager,
    +			IOManager ioManager,
    +			NetworkEnvironment networkEnvironment,
    +			int numberOfSlots,
     			RpcService rpcService,
    -			HighAvailabilityServices haServices,
    -			ResourceID resourceID) {
    +			HighAvailabilityServices haServices) {
     
     		super(rpcService);
     
    -		this.haServices = checkNotNull(haServices);
    +		this.taskManagerConfig = checkNotNull(taskManagerConfig);
     		this.resourceID = checkNotNull(resourceID);
    +		this.connectionInfo = checkNotNull(connectionInfo);
    +		this.memoryManager = checkNotNull(memoryManager);
    +		this.ioManager = checkNotNull(ioManager);
    +		this.networkEnvironment = checkNotNull(networkEnvironment);
    +		this.numberOfSlots = checkNotNull(numberOfSlots);
    +		this.haServices = checkNotNull(haServices);
    +	}
    +
    +	/**
    +	 * Starts and runs the TaskManager.
    +	 * <p/>
    +	 * This method first tries to select the network interface to use for the TaskManager
    +	 * communication. The network interface is used both for the actor communication
    +	 * (coordination) as well as for the data exchange between task managers. Unless
    +	 * the hostname/interface is explicitly configured in the configuration, this
    +	 * method will try out various interfaces and methods to connect to the JobManager
    +	 * and select the one where the connection attempt is successful.
    +	 * <p/>
    +	 * After selecting the network interface, this method brings up an actor system
    +	 * for the TaskManager and its actors, starts the TaskManager's services
    +	 * (library cache, shuffle network stack, ...), and starts the TaskManager itself.
    +	 *
    +	 * @param configuration    The configuration for the TaskManager.
    +	 * @param taskManagerClass The actor class to instantiate.
    +	 *                         Allows to use TaskManager subclasses for example for YARN.
    +	 */
    +	public static void selectNetworkInterfaceAndRunTaskManager(
    +		Configuration configuration,
    +		ResourceID resourceID,
    +		Class<? extends TaskManager> taskManagerClass) throws Exception {
    +
    +		Tuple2<String, Integer> tuple2 = selectNetworkInterfaceAndPort(configuration);
    +
    +		runTaskManager(tuple2._1(), resourceID, tuple2._2(), configuration, taskManagerClass);
    +	}
    +
    +	private static Tuple2<String, Integer> selectNetworkInterfaceAndPort(Configuration configuration)
    +		throws Exception {
    +		String taskManagerHostname = configuration.getString(ConfigConstants.TASK_MANAGER_HOSTNAME_KEY, null);
    +		if (taskManagerHostname != null) {
    +			LOG.info("Using configured hostname/address for TaskManager: " + taskManagerHostname);
    +		} else {
    +			LeaderRetrievalService leaderRetrievalService = LeaderRetrievalUtils.createLeaderRetrievalService(configuration);
    +			FiniteDuration lookupTimeout = AkkaUtils.getLookupTimeout(configuration);
    +
    +			InetAddress taskManagerAddress = LeaderRetrievalUtils.findConnectingAddress(leaderRetrievalService, lookupTimeout);
    +			taskManagerHostname = taskManagerAddress.getHostName();
    +			LOG.info("TaskManager will use hostname/address '{}' ({}) for communication.",
    +				taskManagerHostname, taskManagerAddress.getHostAddress());
    +		}
    +
    +		// if no task manager port has been configured, use 0 (system will pick any free port)
    +		int actorSystemPort = configuration.getInteger(ConfigConstants.TASK_MANAGER_IPC_PORT_KEY, 0);
    +		if (actorSystemPort < 0 || actorSystemPort > 65535) {
    +			throw new IllegalConfigurationException("Invalid value for '" +
    +				ConfigConstants.TASK_MANAGER_IPC_PORT_KEY +
    +				"' (port for the TaskManager actor system) : " + actorSystemPort +
    +				" - Leave config parameter empty or use 0 to let the system choose a port automatically.");
    +		}
    +
    +		return new Tuple2<>(taskManagerHostname, actorSystemPort);
    +	}
    +
    +	/**
    +	 * Starts and runs the TaskManager. Brings up an actor system for the TaskManager and its
    +	 * actors, starts the TaskManager's services (library cache, shuffle network stack, ...),
    +	 * and starts the TaskManager itself.
    +	 * <p/>
    +	 * This method will also spawn a process reaper for the TaskManager (kill the process if
    +	 * the actor fails) and optionally start the JVM memory logging thread.
    +	 *
    +	 * @param taskManagerHostname The hostname/address of the interface where the actor system
    +	 *                            will communicate.
    +	 * @param resourceID          The id of the resource which the task manager will run on.
    +	 * @param actorSystemPort     The port at which the actor system will communicate.
    +	 * @param configuration       The configuration for the TaskManager.
    +	 * @param taskManagerClass    The actor class to instantiate. Allows the use of TaskManager
    +	 *                            subclasses for example for YARN.
    +	 */
    +	private static void runTaskManager(
    +		String taskManagerHostname,
    +		ResourceID resourceID,
    +		int actorSystemPort,
    +		Configuration configuration,
    +		Class<? extends TaskManager> taskManagerClass) throws Exception {
    +
    +		LOG.info("Starting TaskManager");
    +
    +		// Bring up the TaskManager actor system first, bind it to the given address.
    +
    +		LOG.info("Starting TaskManager actor system at " +
    +			NetUtils.hostAndPortToUrlString(taskManagerHostname, actorSystemPort));
    +
    +		ActorSystem taskManagerSystem;
    +		try {
    +			Tuple2<String, Object> address = new Tuple2<String, Object>(taskManagerHostname, actorSystemPort);
    +			Config akkaConfig = AkkaUtils.getAkkaConfig(configuration, new Some<>(address));
    +			LOG.debug("Using akka configuration\n " + akkaConfig);
    +			taskManagerSystem = AkkaUtils.createActorSystem(akkaConfig);
    +		} catch (Throwable t) {
    +			if (t instanceof org.jboss.netty.channel.ChannelException) {
    +				Throwable cause = t.getCause();
    +				if (cause != null && t.getCause() instanceof java.net.BindException) {
    +					String address = NetUtils.hostAndPortToUrlString(taskManagerHostname, actorSystemPort);
    +					throw new IOException("Unable to bind TaskManager actor system to address " +
    +						address + " - " + cause.getMessage(), t);
    +				}
    +			}
    +			throw new Exception("Could not create TaskManager actor system", t);
    +		}
    +
    +		// start all the TaskManager services (network stack,  library cache, ...)
    +		// and the TaskManager actor
    +		try {
    +			LOG.info("Starting TaskManager actor");
    +			ActorRef taskManagerActor = startTaskManagerComponentsAndActor(
    +				configuration,
    +				resourceID,
    +				taskManagerSystem,
    +				taskManagerHostname,
    +				TASK_MANAGER_NAME,
    +				null,
    +				false,
    +				taskManagerClass);
    +
    +			// start a process reaper that watches the JobManager. If the TaskManager actor dies,
    +			// the process reaper will kill the JVM process (to ensure easy failure detection)
    +			LOG.debug("Starting TaskManager process reaper");
    +
    +			taskManagerSystem.actorOf(
    +				Props.create(ProcessReaper.class, taskManagerActor, LOG, RUNTIME_FAILURE_RETURN_CODE),
    +				"TaskManager_Process_Reaper");
    +
    +			// if desired, start the logging daemon that periodically logs the memory usage information
    +			if (LOG.isInfoEnabled() && configuration.getBoolean(
    +				ConfigConstants.TASK_MANAGER_DEBUG_MEMORY_USAGE_START_LOG_THREAD,
    +				ConfigConstants.DEFAULT_TASK_MANAGER_DEBUG_MEMORY_USAGE_START_LOG_THREAD)) {
    +				LOG.info("Starting periodic memory usage logger");
    +
    +				long interval = configuration.getLong(
    +					ConfigConstants.TASK_MANAGER_DEBUG_MEMORY_USAGE_LOG_INTERVAL_MS,
    +					ConfigConstants.DEFAULT_TASK_MANAGER_DEBUG_MEMORY_USAGE_LOG_INTERVAL_MS);
    +
    +				MemoryLogger logger = new MemoryLogger(LOG, interval, taskManagerSystem);
    +				logger.start();
    +			}
    +
    +			// block until everything is done
    +			taskManagerSystem.awaitTermination();
    +		} catch (Throwable t) {
    +			LOG.error("Error while starting up taskManager", t);
    +			try {
    +				taskManagerSystem.shutdown();
    +			} catch (Throwable tt) {
    +				LOG.warn("Could not cleanly shut down actor system", tt);
    +			}
    +			throw t;
    +		}
    +	}
    +
    +	// --------------------------------------------------------------------------
    +	//  Starting and running the TaskManager
    +	// --------------------------------------------------------------------------
    +
    +	/**
    +	 * @param configuration                 The configuration for the TaskManager.
    +	 * @param resourceID                    The id of the resource which the task manager will run on.
    +	 * @param actorSystem                  The actor system that should run the TaskManager actor.
    +	 * @param taskManagerHostname       The hostname/address that describes the TaskManager's data location.
    +	 * @param taskManagerActorName      Optionally the name of the TaskManager actor. If none is given,
    +	 *                                      the actor will use a random name.
    +	 * @param leaderRetrievalService        Optionally, a leader retrieval service can be provided. If none is given,
    +	 *                                      then a LeaderRetrievalService is constructed from the configuration.
    +	 * @param localTaskManagerCommunication If true, the TaskManager will not initiate the TCP network stack.
    +	 * @param taskManagerClass      The class of the TaskManager actor. May be used to give
    +	 *                                      subclasses that understand additional actor messages.
    +	 * @return An ActorRef to the TaskManager actor.
    +	 * @throws org.apache.flink.configuration.IllegalConfigurationException     Thrown, if the given config contains illegal values.
    +	 * @throws java.io.IOException      Thrown, if any of the I/O components (such as buffer pools,
    +	 *                                       I/O manager, ...) cannot be properly started.
    +	 * @throws java.lang.Exception      Thrown is some other error occurs while parsing the configuration
    +	 *                                      or starting the TaskManager components.
    +	 */
    +	public static ActorRef startTaskManagerComponentsAndActor(
    +		Configuration configuration,
    +		ResourceID resourceID,
    +		ActorSystem actorSystem,
    +		String taskManagerHostname,
    +		String taskManagerActorName,
    +		LeaderRetrievalService leaderRetrievalService,
    +		boolean localTaskManagerCommunication,
    +		Class<? extends TaskManager> taskManagerClass) throws Exception {
    +
    +		Tuple4<TaskManagerConfiguration, NetworkEnvironmentConfiguration, InstanceConnectionInfo, MemoryType> tuple4
    +			= parseTaskManagerConfiguration(configuration, taskManagerHostname, localTaskManagerCommunication);
    +
    +		TaskManagerConfiguration taskManagerConfig = tuple4._1();
    +		NetworkEnvironmentConfiguration netConfig = tuple4._2();
    +		InstanceConnectionInfo connectionInfo = tuple4._3();
    +		MemoryType memType = tuple4._4();
    +
    +		// pre-start checks
    +		checkTempDirs(taskManagerConfig.tmpDirPaths());
    +
    +		ExecutionContext executionContext = ExecutionContexts$.MODULE$.fromExecutor(new ForkJoinPool());
    +
    +		// we start the network first, to make sure it can allocate its buffers first
    +		NetworkEnvironment network = new NetworkEnvironment(executionContext, taskManagerConfig.timeout(), netConfig);
    +
    +		// computing the amount of memory to use depends on how much memory is available
    +		// it strictly needs to happen AFTER the network stack has been initialized
    +
    +		// check if a value has been configured
    +		long configuredMemory = configuration.getLong(ConfigConstants.TASK_MANAGER_MEMORY_SIZE_KEY, -1L);
    +		checkConfigParameter(configuredMemory == -1 || configuredMemory > 0, configuredMemory,
    +			ConfigConstants.TASK_MANAGER_MEMORY_SIZE_KEY,
    +			"MemoryManager needs at least one MB of memory. " +
    +				"If you leave this config parameter empty, the system automatically " +
    +				"pick a fraction of the available memory.");
    +
    +		long memorySize;
    --- End diff --
    
    The final statement has added


---
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.
---

[GitHub] flink pull request #2400: [FLINK-4363] Implement TaskManager basic startup o...

Posted by mxm <gi...@git.apache.org>.
Github user mxm commented on a diff in the pull request:

    https://github.com/apache/flink/pull/2400#discussion_r75647928
  
    --- Diff: flink-runtime/src/main/java/org/apache/flink/runtime/rpc/taskexecutor/TaskExecutor.java ---
    @@ -35,27 +79,634 @@
      */
     public class TaskExecutor extends RpcEndpoint<TaskExecutorGateway> {
     
    -	/** The unique resource ID of this TaskExecutor */
    +	private static final Logger LOG = LoggerFactory.getLogger(TaskExecutor.class);
    +
    +	/** Return code for critical errors during the runtime */
    +	private static final int RUNTIME_FAILURE_RETURN_CODE = 2;
    +
    +	/** The name of the TaskManager actor */
    +	private static final String TASK_MANAGER_NAME = "taskmanager";
    +
    +	/** The unique resource ID of this TaskManager */
     	private final ResourceID resourceID;
     
     	/** The access to the leader election and metadata storage services */
     	private final HighAvailabilityServices haServices;
     
    -	// --------- resource manager --------
    +	/** The task manager configuration */
    +	private final TaskManagerConfiguration taskManagerConfig;
    +
    +	/** The connection information of the task manager */
    +	private final InstanceConnectionInfo connectionInfo;
    +
    +	/** The I/O manager component in the task manager */
    +	private final IOManager ioManager;
    +
    +	/** The memory manager component in the task manager */
    +	private final MemoryManager memoryManager;
    +
    +	/** The network component in the task manager */
    +	private final NetworkEnvironment networkEnvironment;
    +
    +	/** The number of slots in the task manager, should be 1 for YARN */
    +	private final int numberOfSlots;
     
    +	// --------- resource manager --------
     	private TaskExecutorToResourceManagerConnection resourceManagerConnection;
     
     	// ------------------------------------------------------------------------
     
     	public TaskExecutor(
    +			TaskManagerConfiguration taskManagerConfig,
    +			ResourceID resourceID,
    +			InstanceConnectionInfo connectionInfo,
    +			MemoryManager memoryManager,
    +			IOManager ioManager,
    +			NetworkEnvironment networkEnvironment,
    +			int numberOfSlots,
     			RpcService rpcService,
    -			HighAvailabilityServices haServices,
    -			ResourceID resourceID) {
    +			HighAvailabilityServices haServices) {
     
     		super(rpcService);
     
    -		this.haServices = checkNotNull(haServices);
    +		this.taskManagerConfig = checkNotNull(taskManagerConfig);
     		this.resourceID = checkNotNull(resourceID);
    +		this.connectionInfo = checkNotNull(connectionInfo);
    +		this.memoryManager = checkNotNull(memoryManager);
    +		this.ioManager = checkNotNull(ioManager);
    +		this.networkEnvironment = checkNotNull(networkEnvironment);
    +		this.numberOfSlots = checkNotNull(numberOfSlots);
    +		this.haServices = checkNotNull(haServices);
    +	}
    +
    +	/**
    +	 * Starts and runs the TaskManager.
    +	 * <p/>
    +	 * This method first tries to select the network interface to use for the TaskManager
    +	 * communication. The network interface is used both for the actor communication
    +	 * (coordination) as well as for the data exchange between task managers. Unless
    +	 * the hostname/interface is explicitly configured in the configuration, this
    +	 * method will try out various interfaces and methods to connect to the JobManager
    +	 * and select the one where the connection attempt is successful.
    +	 * <p/>
    +	 * After selecting the network interface, this method brings up an actor system
    +	 * for the TaskManager and its actors, starts the TaskManager's services
    +	 * (library cache, shuffle network stack, ...), and starts the TaskManager itself.
    +	 *
    +	 * @param configuration    The configuration for the TaskManager.
    +	 * @param taskManagerClass The actor class to instantiate.
    +	 *                         Allows to use TaskManager subclasses for example for YARN.
    +	 */
    +	public static void selectNetworkInterfaceAndRunTaskManager(
    +		Configuration configuration,
    +		ResourceID resourceID,
    +		Class<? extends TaskManager> taskManagerClass) throws Exception {
    +
    +		Tuple2<String, Integer> tuple2 = selectNetworkInterfaceAndPort(configuration);
    +
    +		runTaskManager(tuple2._1(), resourceID, tuple2._2(), configuration, taskManagerClass);
    +	}
    +
    +	private static Tuple2<String, Integer> selectNetworkInterfaceAndPort(Configuration configuration)
    +		throws Exception {
    +		String taskManagerHostname = configuration.getString(ConfigConstants.TASK_MANAGER_HOSTNAME_KEY, null);
    +		if (taskManagerHostname != null) {
    +			LOG.info("Using configured hostname/address for TaskManager: " + taskManagerHostname);
    +		} else {
    +			LeaderRetrievalService leaderRetrievalService = LeaderRetrievalUtils.createLeaderRetrievalService(configuration);
    +			FiniteDuration lookupTimeout = AkkaUtils.getLookupTimeout(configuration);
    +
    +			InetAddress taskManagerAddress = LeaderRetrievalUtils.findConnectingAddress(leaderRetrievalService, lookupTimeout);
    +			taskManagerHostname = taskManagerAddress.getHostName();
    +			LOG.info("TaskManager will use hostname/address '{}' ({}) for communication.",
    +				taskManagerHostname, taskManagerAddress.getHostAddress());
    +		}
    +
    +		// if no task manager port has been configured, use 0 (system will pick any free port)
    +		int actorSystemPort = configuration.getInteger(ConfigConstants.TASK_MANAGER_IPC_PORT_KEY, 0);
    +		if (actorSystemPort < 0 || actorSystemPort > 65535) {
    +			throw new IllegalConfigurationException("Invalid value for '" +
    +				ConfigConstants.TASK_MANAGER_IPC_PORT_KEY +
    +				"' (port for the TaskManager actor system) : " + actorSystemPort +
    +				" - Leave config parameter empty or use 0 to let the system choose a port automatically.");
    +		}
    +
    +		return new Tuple2<>(taskManagerHostname, actorSystemPort);
    +	}
    +
    +	/**
    +	 * Starts and runs the TaskManager. Brings up an actor system for the TaskManager and its
    +	 * actors, starts the TaskManager's services (library cache, shuffle network stack, ...),
    +	 * and starts the TaskManager itself.
    +	 * <p/>
    +	 * This method will also spawn a process reaper for the TaskManager (kill the process if
    +	 * the actor fails) and optionally start the JVM memory logging thread.
    +	 *
    +	 * @param taskManagerHostname The hostname/address of the interface where the actor system
    +	 *                            will communicate.
    +	 * @param resourceID          The id of the resource which the task manager will run on.
    +	 * @param actorSystemPort     The port at which the actor system will communicate.
    +	 * @param configuration       The configuration for the TaskManager.
    +	 * @param taskManagerClass    The actor class to instantiate. Allows the use of TaskManager
    +	 *                            subclasses for example for YARN.
    +	 */
    +	private static void runTaskManager(
    +		String taskManagerHostname,
    +		ResourceID resourceID,
    +		int actorSystemPort,
    +		Configuration configuration,
    +		Class<? extends TaskManager> taskManagerClass) throws Exception {
    +
    +		LOG.info("Starting TaskManager");
    +
    +		// Bring up the TaskManager actor system first, bind it to the given address.
    +
    +		LOG.info("Starting TaskManager actor system at " +
    +			NetUtils.hostAndPortToUrlString(taskManagerHostname, actorSystemPort));
    +
    +		ActorSystem taskManagerSystem;
    +		try {
    +			Tuple2<String, Object> address = new Tuple2<String, Object>(taskManagerHostname, actorSystemPort);
    +			Config akkaConfig = AkkaUtils.getAkkaConfig(configuration, new Some<>(address));
    +			LOG.debug("Using akka configuration\n " + akkaConfig);
    +			taskManagerSystem = AkkaUtils.createActorSystem(akkaConfig);
    +		} catch (Throwable t) {
    +			if (t instanceof org.jboss.netty.channel.ChannelException) {
    +				Throwable cause = t.getCause();
    +				if (cause != null && t.getCause() instanceof java.net.BindException) {
    +					String address = NetUtils.hostAndPortToUrlString(taskManagerHostname, actorSystemPort);
    +					throw new IOException("Unable to bind TaskManager actor system to address " +
    +						address + " - " + cause.getMessage(), t);
    +				}
    +			}
    +			throw new Exception("Could not create TaskManager actor system", t);
    +		}
    +
    +		// start all the TaskManager services (network stack,  library cache, ...)
    +		// and the TaskManager actor
    +		try {
    +			LOG.info("Starting TaskManager actor");
    +			ActorRef taskManagerActor = startTaskManagerComponentsAndActor(
    +				configuration,
    +				resourceID,
    +				taskManagerSystem,
    +				taskManagerHostname,
    +				TASK_MANAGER_NAME,
    +				null,
    +				false,
    +				taskManagerClass);
    +
    +			// start a process reaper that watches the JobManager. If the TaskManager actor dies,
    +			// the process reaper will kill the JVM process (to ensure easy failure detection)
    +			LOG.debug("Starting TaskManager process reaper");
    +
    +			taskManagerSystem.actorOf(
    +				Props.create(ProcessReaper.class, taskManagerActor, LOG, RUNTIME_FAILURE_RETURN_CODE),
    +				"TaskManager_Process_Reaper");
    +
    +			// if desired, start the logging daemon that periodically logs the memory usage information
    +			if (LOG.isInfoEnabled() && configuration.getBoolean(
    +				ConfigConstants.TASK_MANAGER_DEBUG_MEMORY_USAGE_START_LOG_THREAD,
    +				ConfigConstants.DEFAULT_TASK_MANAGER_DEBUG_MEMORY_USAGE_START_LOG_THREAD)) {
    +				LOG.info("Starting periodic memory usage logger");
    +
    +				long interval = configuration.getLong(
    +					ConfigConstants.TASK_MANAGER_DEBUG_MEMORY_USAGE_LOG_INTERVAL_MS,
    +					ConfigConstants.DEFAULT_TASK_MANAGER_DEBUG_MEMORY_USAGE_LOG_INTERVAL_MS);
    +
    +				MemoryLogger logger = new MemoryLogger(LOG, interval, taskManagerSystem);
    +				logger.start();
    +			}
    +
    +			// block until everything is done
    +			taskManagerSystem.awaitTermination();
    +		} catch (Throwable t) {
    +			LOG.error("Error while starting up taskManager", t);
    +			try {
    +				taskManagerSystem.shutdown();
    +			} catch (Throwable tt) {
    +				LOG.warn("Could not cleanly shut down actor system", tt);
    +			}
    +			throw t;
    +		}
    +	}
    +
    +	// --------------------------------------------------------------------------
    +	//  Starting and running the TaskManager
    +	// --------------------------------------------------------------------------
    +
    +	/**
    +	 * @param configuration                 The configuration for the TaskManager.
    +	 * @param resourceID                    The id of the resource which the task manager will run on.
    +	 * @param actorSystem                  The actor system that should run the TaskManager actor.
    +	 * @param taskManagerHostname       The hostname/address that describes the TaskManager's data location.
    +	 * @param taskManagerActorName      Optionally the name of the TaskManager actor. If none is given,
    +	 *                                      the actor will use a random name.
    +	 * @param leaderRetrievalService        Optionally, a leader retrieval service can be provided. If none is given,
    +	 *                                      then a LeaderRetrievalService is constructed from the configuration.
    +	 * @param localTaskManagerCommunication If true, the TaskManager will not initiate the TCP network stack.
    +	 * @param taskManagerClass      The class of the TaskManager actor. May be used to give
    +	 *                                      subclasses that understand additional actor messages.
    +	 * @return An ActorRef to the TaskManager actor.
    +	 * @throws org.apache.flink.configuration.IllegalConfigurationException     Thrown, if the given config contains illegal values.
    +	 * @throws java.io.IOException      Thrown, if any of the I/O components (such as buffer pools,
    +	 *                                       I/O manager, ...) cannot be properly started.
    +	 * @throws java.lang.Exception      Thrown is some other error occurs while parsing the configuration
    +	 *                                      or starting the TaskManager components.
    +	 */
    +	public static ActorRef startTaskManagerComponentsAndActor(
    +		Configuration configuration,
    +		ResourceID resourceID,
    +		ActorSystem actorSystem,
    +		String taskManagerHostname,
    +		String taskManagerActorName,
    +		LeaderRetrievalService leaderRetrievalService,
    +		boolean localTaskManagerCommunication,
    +		Class<? extends TaskManager> taskManagerClass) throws Exception {
    +
    +		Tuple4<TaskManagerConfiguration, NetworkEnvironmentConfiguration, InstanceConnectionInfo, MemoryType> tuple4
    +			= parseTaskManagerConfiguration(configuration, taskManagerHostname, localTaskManagerCommunication);
    +
    +		TaskManagerConfiguration taskManagerConfig = tuple4._1();
    +		NetworkEnvironmentConfiguration netConfig = tuple4._2();
    +		InstanceConnectionInfo connectionInfo = tuple4._3();
    +		MemoryType memType = tuple4._4();
    +
    +		// pre-start checks
    +		checkTempDirs(taskManagerConfig.tmpDirPaths());
    +
    +		ExecutionContext executionContext = ExecutionContexts$.MODULE$.fromExecutor(new ForkJoinPool());
    +
    +		// we start the network first, to make sure it can allocate its buffers first
    +		NetworkEnvironment network = new NetworkEnvironment(executionContext, taskManagerConfig.timeout(), netConfig);
    +
    +		// computing the amount of memory to use depends on how much memory is available
    +		// it strictly needs to happen AFTER the network stack has been initialized
    +
    +		// check if a value has been configured
    +		long configuredMemory = configuration.getLong(ConfigConstants.TASK_MANAGER_MEMORY_SIZE_KEY, -1L);
    +		checkConfigParameter(configuredMemory == -1 || configuredMemory > 0, configuredMemory,
    +			ConfigConstants.TASK_MANAGER_MEMORY_SIZE_KEY,
    +			"MemoryManager needs at least one MB of memory. " +
    +				"If you leave this config parameter empty, the system automatically " +
    +				"pick a fraction of the available memory.");
    +
    +		long memorySize;
    +		boolean preAllocateMemory = configuration.getBoolean(
    +			ConfigConstants.TASK_MANAGER_MEMORY_PRE_ALLOCATE_KEY,
    +			ConfigConstants.DEFAULT_TASK_MANAGER_MEMORY_PRE_ALLOCATE);
    +		if (configuredMemory > 0) {
    +			if (preAllocateMemory) {
    +				LOG.info("Using {} MB for managed memory." , configuredMemory);
    +			} else {
    +				LOG.info("Limiting managed memory to {} MB, memory will be allocated lazily." , configuredMemory);
    +			}
    +			memorySize = configuredMemory << 20; // megabytes to bytes
    +		} else {
    +			float fraction = configuration.getFloat(
    +				ConfigConstants.TASK_MANAGER_MEMORY_FRACTION_KEY,
    +				ConfigConstants.DEFAULT_MEMORY_MANAGER_MEMORY_FRACTION);
    +			checkConfigParameter(fraction > 0.0f && fraction < 1.0f, fraction,
    +				ConfigConstants.TASK_MANAGER_MEMORY_FRACTION_KEY,
    +				"MemoryManager fraction of the free memory must be between 0.0 and 1.0");
    +
    +			if (memType == MemoryType.HEAP) {
    +				long relativeMemSize = (long) (EnvironmentInformation.getSizeOfFreeHeapMemoryWithDefrag() * fraction);
    +				if (preAllocateMemory) {
    +					LOG.info("Using {} of the currently free heap space for managed heap memory ({} MB)." ,
    +						fraction , relativeMemSize >> 20);
    +				} else {
    +					LOG.info("Limiting managed memory to {} of the currently free heap space ({} MB), " +
    +						"memory will be allocated lazily." , fraction , relativeMemSize >> 20);
    +				}
    +				memorySize = relativeMemSize;
    +			} else if (memType == MemoryType.OFF_HEAP) {
    +				// The maximum heap memory has been adjusted according to the fraction
    +				long maxMemory = EnvironmentInformation.getMaxJvmHeapMemory();
    +				long directMemorySize = (long) (maxMemory / (1.0 - fraction) * fraction);
    +				if (preAllocateMemory) {
    +					LOG.info("Using {} of the maximum memory size for managed off-heap memory ({} MB)." ,
    +						fraction, directMemorySize >> 20);
    +				} else {
    +					LOG.info("Limiting managed memory to {} of the maximum memory size ({} MB)," +
    +						" memory will be allocated lazily.", fraction, directMemorySize >> 20);
    +				}
    +				memorySize = directMemorySize;
    +			} else {
    +				throw new RuntimeException("No supported memory type detected.");
    +			}
    +		}
    +
    +		// now start the memory manager
    +		MemoryManager memoryManager;
    +		try {
    +			memoryManager = new MemoryManager(
    +				memorySize,
    +				taskManagerConfig.numberOfSlots(),
    +				netConfig.networkBufferSize(),
    +				memType,
    +				preAllocateMemory);
    +		} catch (OutOfMemoryError e) {
    +			if (memType == MemoryType.HEAP) {
    +				throw new Exception("OutOfMemory error (" + e.getMessage() +
    +					") while allocating the TaskManager heap memory (" + memorySize + " bytes).", e);
    +			} else if (memType == MemoryType.OFF_HEAP) {
    +				throw new Exception("OutOfMemory error (" + e.getMessage() +
    +					") while allocating the TaskManager off-heap memory (" + memorySize +
    +					" bytes).Try increasing the maximum direct memory (-XX:MaxDirectMemorySize)", e);
    +			} else {
    +				throw e;
    +			}
    +		}
    +
    +		// start the I/O manager, it will create some temp directories.
    +		IOManager ioManager = new IOManagerAsync(taskManagerConfig.tmpDirPaths());
    +
    +		if (leaderRetrievalService == null){
    +			leaderRetrievalService = LeaderRetrievalUtils.createLeaderRetrievalService(configuration);
    +		}
    +
    +		// create the actor properties (which define the actor constructor parameters)
    +		Props tmProps = Props.create(
    +			taskManagerClass,
    +			taskManagerConfig,
    +			resourceID,
    +			connectionInfo,
    +			memoryManager,
    +			ioManager,
    +			network,
    +			taskManagerConfig.numberOfSlots(),
    +			leaderRetrievalService);
    +
    +		ActorRef taskManagerActorRef;
    +		if (taskManagerActorName != null && !taskManagerActorName.equals("")) {
    +			taskManagerActorRef = actorSystem.actorOf(tmProps, taskManagerActorName);
    +		} else {
    +			taskManagerActorRef = actorSystem.actorOf(tmProps);
    +		}
    +
    +		return taskManagerActorRef;
    +	}
    +
    +	// --------------------------------------------------------------------------
    +	//  Parsing and checking the TaskManager Configuration
    +	// --------------------------------------------------------------------------
    +
    +	/**
    +	 * Utility method to extract TaskManager config parameters from the configuration and to
    +	 * sanity check them.
    +	 *
    +	 * @param configuration                 The configuration.
    +	 * @param taskManagerHostname           The host name under which the TaskManager communicates.
    +	 * @param localTaskManagerCommunication True, to skip initializing the network stack.
    +	 *                                      Use only in cases where only one task manager runs.
    +	 * @return A tuple (TaskManagerConfiguration, network configuration,
    +	 * InstanceConnectionInfo, JobManager actor Akka URL).
    +	 */
    +	private static Tuple4<TaskManagerConfiguration, NetworkEnvironmentConfiguration, InstanceConnectionInfo, MemoryType>
    +		parseTaskManagerConfiguration(Configuration configuration, String taskManagerHostname, boolean localTaskManagerCommunication)
    +		throws Exception {
    +
    +		// ------- read values from the config and check them ---------
    +		//                      (a lot of them)
    +
    +		// ----> hosts / ports for communication and data exchange
    +
    +		int dataport = configuration.getInteger(ConfigConstants.TASK_MANAGER_DATA_PORT_KEY,
    +			ConfigConstants.DEFAULT_TASK_MANAGER_DATA_PORT);
    +		if (dataport == 0) {
    +			dataport = NetUtils.getAvailablePort();
    +		}
    +		checkConfigParameter(dataport > 0, dataport, ConfigConstants.TASK_MANAGER_DATA_PORT_KEY,
    +			"Leave config parameter empty or use 0 to let the system choose a port automatically.");
    +
    +		InetAddress taskManagerAddress = InetAddress.getByName(taskManagerHostname);
    +		InstanceConnectionInfo connectionInfo = new InstanceConnectionInfo(taskManagerAddress, dataport);
    +
    +		// ----> memory / network stack (shuffles/broadcasts), task slots, temp directories
    +
    +		// we need this because many configs have been written with a "-1" entry
    +		int slots = configuration.getInteger(ConfigConstants.TASK_MANAGER_NUM_TASK_SLOTS, 1);
    +		if (slots == -1) {
    +			slots = 1;
    +		}
    +		checkConfigParameter(slots >= 1, slots, ConfigConstants.TASK_MANAGER_NUM_TASK_SLOTS,
    +			"Number of task slots must be at least one.");
    +
    +		int numNetworkBuffers = configuration.getInteger(
    +			ConfigConstants.TASK_MANAGER_NETWORK_NUM_BUFFERS_KEY,
    +			ConfigConstants.DEFAULT_TASK_MANAGER_NETWORK_NUM_BUFFERS);
    +		checkConfigParameter(numNetworkBuffers > 0, numNetworkBuffers,
    +			ConfigConstants.TASK_MANAGER_NETWORK_NUM_BUFFERS_KEY, "");
    +
    +		int pageSize = configuration.getInteger(
    +			ConfigConstants.TASK_MANAGER_MEMORY_SEGMENT_SIZE_KEY,
    +			ConfigConstants.DEFAULT_TASK_MANAGER_MEMORY_SEGMENT_SIZE);
    +		// check page size of for minimum size
    +		checkConfigParameter(pageSize >= MemoryManager.MIN_PAGE_SIZE, pageSize,
    +			ConfigConstants.TASK_MANAGER_MEMORY_SEGMENT_SIZE_KEY,
    +			"Minimum memory segment size is " + MemoryManager.MIN_PAGE_SIZE);
    +		// check page size for power of two
    +		checkConfigParameter(MathUtils.isPowerOf2(pageSize), pageSize,
    +			ConfigConstants.TASK_MANAGER_MEMORY_SEGMENT_SIZE_KEY,
    +			"Memory segment size must be a power of 2.");
    +
    +		// check whether we use heap or off-heap memory
    +		MemoryType memType;
    +		if (configuration.getBoolean(ConfigConstants.TASK_MANAGER_MEMORY_OFF_HEAP_KEY, false)) {
    +			memType = MemoryType.OFF_HEAP;
    +		} else {
    +			memType = MemoryType.HEAP;
    +		}
    +
    +		// initialize the memory segment factory accordingly
    +		if (memType == MemoryType.HEAP) {
    +			if (!MemorySegmentFactory.isInitialized()) {
    +				MemorySegmentFactory.initializeFactory(HeapMemorySegment.FACTORY);
    +			} else if (MemorySegmentFactory.getFactory() != HeapMemorySegment.FACTORY) {
    +				throw new Exception("Memory type is set to heap memory, but memory segment " +
    +					"factory has been initialized for off-heap memory segments");
    +			}
    +		} else {
    +			if (!MemorySegmentFactory.isInitialized()) {
    +				MemorySegmentFactory.initializeFactory(HybridMemorySegment.FACTORY);
    +			} else if (MemorySegmentFactory.getFactory() != HybridMemorySegment.FACTORY) {
    +				throw new Exception("Memory type is set to off-heap memory, but memory segment " +
    +					"factory has been initialized for heap memory segments");
    +			}
    +		}
    +
    +		String[] tmpDirs = configuration.getString(
    +			ConfigConstants.TASK_MANAGER_TMP_DIR_KEY,
    +			ConfigConstants.DEFAULT_TASK_MANAGER_TMP_PATH).split(",|" + File.pathSeparator);
    +
    +		NettyConfig nettyConfig = null;
    +		if (!localTaskManagerCommunication) {
    +			nettyConfig = new NettyConfig(connectionInfo.address(), connectionInfo.dataPort(), pageSize, slots, configuration);
    +		}
    +
    +		// Default spill I/O mode for intermediate results
    +		String syncOrAsync = configuration.getString(
    +			ConfigConstants.TASK_MANAGER_NETWORK_DEFAULT_IO_MODE,
    +			ConfigConstants.DEFAULT_TASK_MANAGER_NETWORK_DEFAULT_IO_MODE);
    +
    +		IOMode ioMode;
    +		if (syncOrAsync == "async") {
    +			ioMode = IOManager.IOMode.ASYNC;
    +		} else {
    +			ioMode = IOManager.IOMode.SYNC;
    +		}
    +
    +		NetworkEnvironmentConfiguration networkConfig = new NetworkEnvironmentConfiguration(
    +			numNetworkBuffers,
    +			pageSize,
    +			memType,
    +			ioMode,
    +			nettyConfig == null ? Option.<NettyConfig>empty() : new Some<>(nettyConfig),
    +			new Tuple2<>(500, 3000));
    +
    +		// ----> timeouts, library caching, profiling
    +
    +		FiniteDuration timeout;
    --- End diff --
    
    `final`


---
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.
---

[GitHub] flink pull request #2400: [FLINK-4363] Implement TaskManager basic startup o...

Posted by mxm <gi...@git.apache.org>.
Github user mxm commented on a diff in the pull request:

    https://github.com/apache/flink/pull/2400#discussion_r75647950
  
    --- Diff: flink-runtime/src/main/java/org/apache/flink/runtime/rpc/taskexecutor/TaskExecutor.java ---
    @@ -35,27 +79,634 @@
      */
     public class TaskExecutor extends RpcEndpoint<TaskExecutorGateway> {
     
    -	/** The unique resource ID of this TaskExecutor */
    +	private static final Logger LOG = LoggerFactory.getLogger(TaskExecutor.class);
    +
    +	/** Return code for critical errors during the runtime */
    +	private static final int RUNTIME_FAILURE_RETURN_CODE = 2;
    +
    +	/** The name of the TaskManager actor */
    +	private static final String TASK_MANAGER_NAME = "taskmanager";
    +
    +	/** The unique resource ID of this TaskManager */
     	private final ResourceID resourceID;
     
     	/** The access to the leader election and metadata storage services */
     	private final HighAvailabilityServices haServices;
     
    -	// --------- resource manager --------
    +	/** The task manager configuration */
    +	private final TaskManagerConfiguration taskManagerConfig;
    +
    +	/** The connection information of the task manager */
    +	private final InstanceConnectionInfo connectionInfo;
    +
    +	/** The I/O manager component in the task manager */
    +	private final IOManager ioManager;
    +
    +	/** The memory manager component in the task manager */
    +	private final MemoryManager memoryManager;
    +
    +	/** The network component in the task manager */
    +	private final NetworkEnvironment networkEnvironment;
    +
    +	/** The number of slots in the task manager, should be 1 for YARN */
    +	private final int numberOfSlots;
     
    +	// --------- resource manager --------
     	private TaskExecutorToResourceManagerConnection resourceManagerConnection;
     
     	// ------------------------------------------------------------------------
     
     	public TaskExecutor(
    +			TaskManagerConfiguration taskManagerConfig,
    +			ResourceID resourceID,
    +			InstanceConnectionInfo connectionInfo,
    +			MemoryManager memoryManager,
    +			IOManager ioManager,
    +			NetworkEnvironment networkEnvironment,
    +			int numberOfSlots,
     			RpcService rpcService,
    -			HighAvailabilityServices haServices,
    -			ResourceID resourceID) {
    +			HighAvailabilityServices haServices) {
     
     		super(rpcService);
     
    -		this.haServices = checkNotNull(haServices);
    +		this.taskManagerConfig = checkNotNull(taskManagerConfig);
     		this.resourceID = checkNotNull(resourceID);
    +		this.connectionInfo = checkNotNull(connectionInfo);
    +		this.memoryManager = checkNotNull(memoryManager);
    +		this.ioManager = checkNotNull(ioManager);
    +		this.networkEnvironment = checkNotNull(networkEnvironment);
    +		this.numberOfSlots = checkNotNull(numberOfSlots);
    +		this.haServices = checkNotNull(haServices);
    +	}
    +
    +	/**
    +	 * Starts and runs the TaskManager.
    +	 * <p/>
    +	 * This method first tries to select the network interface to use for the TaskManager
    +	 * communication. The network interface is used both for the actor communication
    +	 * (coordination) as well as for the data exchange between task managers. Unless
    +	 * the hostname/interface is explicitly configured in the configuration, this
    +	 * method will try out various interfaces and methods to connect to the JobManager
    +	 * and select the one where the connection attempt is successful.
    +	 * <p/>
    +	 * After selecting the network interface, this method brings up an actor system
    +	 * for the TaskManager and its actors, starts the TaskManager's services
    +	 * (library cache, shuffle network stack, ...), and starts the TaskManager itself.
    +	 *
    +	 * @param configuration    The configuration for the TaskManager.
    +	 * @param taskManagerClass The actor class to instantiate.
    +	 *                         Allows to use TaskManager subclasses for example for YARN.
    +	 */
    +	public static void selectNetworkInterfaceAndRunTaskManager(
    +		Configuration configuration,
    +		ResourceID resourceID,
    +		Class<? extends TaskManager> taskManagerClass) throws Exception {
    +
    +		Tuple2<String, Integer> tuple2 = selectNetworkInterfaceAndPort(configuration);
    +
    +		runTaskManager(tuple2._1(), resourceID, tuple2._2(), configuration, taskManagerClass);
    +	}
    +
    +	private static Tuple2<String, Integer> selectNetworkInterfaceAndPort(Configuration configuration)
    +		throws Exception {
    +		String taskManagerHostname = configuration.getString(ConfigConstants.TASK_MANAGER_HOSTNAME_KEY, null);
    +		if (taskManagerHostname != null) {
    +			LOG.info("Using configured hostname/address for TaskManager: " + taskManagerHostname);
    +		} else {
    +			LeaderRetrievalService leaderRetrievalService = LeaderRetrievalUtils.createLeaderRetrievalService(configuration);
    +			FiniteDuration lookupTimeout = AkkaUtils.getLookupTimeout(configuration);
    +
    +			InetAddress taskManagerAddress = LeaderRetrievalUtils.findConnectingAddress(leaderRetrievalService, lookupTimeout);
    +			taskManagerHostname = taskManagerAddress.getHostName();
    +			LOG.info("TaskManager will use hostname/address '{}' ({}) for communication.",
    +				taskManagerHostname, taskManagerAddress.getHostAddress());
    +		}
    +
    +		// if no task manager port has been configured, use 0 (system will pick any free port)
    +		int actorSystemPort = configuration.getInteger(ConfigConstants.TASK_MANAGER_IPC_PORT_KEY, 0);
    +		if (actorSystemPort < 0 || actorSystemPort > 65535) {
    +			throw new IllegalConfigurationException("Invalid value for '" +
    +				ConfigConstants.TASK_MANAGER_IPC_PORT_KEY +
    +				"' (port for the TaskManager actor system) : " + actorSystemPort +
    +				" - Leave config parameter empty or use 0 to let the system choose a port automatically.");
    +		}
    +
    +		return new Tuple2<>(taskManagerHostname, actorSystemPort);
    +	}
    +
    +	/**
    +	 * Starts and runs the TaskManager. Brings up an actor system for the TaskManager and its
    +	 * actors, starts the TaskManager's services (library cache, shuffle network stack, ...),
    +	 * and starts the TaskManager itself.
    +	 * <p/>
    +	 * This method will also spawn a process reaper for the TaskManager (kill the process if
    +	 * the actor fails) and optionally start the JVM memory logging thread.
    +	 *
    +	 * @param taskManagerHostname The hostname/address of the interface where the actor system
    +	 *                            will communicate.
    +	 * @param resourceID          The id of the resource which the task manager will run on.
    +	 * @param actorSystemPort     The port at which the actor system will communicate.
    +	 * @param configuration       The configuration for the TaskManager.
    +	 * @param taskManagerClass    The actor class to instantiate. Allows the use of TaskManager
    +	 *                            subclasses for example for YARN.
    +	 */
    +	private static void runTaskManager(
    +		String taskManagerHostname,
    +		ResourceID resourceID,
    +		int actorSystemPort,
    +		Configuration configuration,
    +		Class<? extends TaskManager> taskManagerClass) throws Exception {
    +
    +		LOG.info("Starting TaskManager");
    +
    +		// Bring up the TaskManager actor system first, bind it to the given address.
    +
    +		LOG.info("Starting TaskManager actor system at " +
    +			NetUtils.hostAndPortToUrlString(taskManagerHostname, actorSystemPort));
    +
    +		ActorSystem taskManagerSystem;
    +		try {
    +			Tuple2<String, Object> address = new Tuple2<String, Object>(taskManagerHostname, actorSystemPort);
    +			Config akkaConfig = AkkaUtils.getAkkaConfig(configuration, new Some<>(address));
    +			LOG.debug("Using akka configuration\n " + akkaConfig);
    +			taskManagerSystem = AkkaUtils.createActorSystem(akkaConfig);
    +		} catch (Throwable t) {
    +			if (t instanceof org.jboss.netty.channel.ChannelException) {
    +				Throwable cause = t.getCause();
    +				if (cause != null && t.getCause() instanceof java.net.BindException) {
    +					String address = NetUtils.hostAndPortToUrlString(taskManagerHostname, actorSystemPort);
    +					throw new IOException("Unable to bind TaskManager actor system to address " +
    +						address + " - " + cause.getMessage(), t);
    +				}
    +			}
    +			throw new Exception("Could not create TaskManager actor system", t);
    +		}
    +
    +		// start all the TaskManager services (network stack,  library cache, ...)
    +		// and the TaskManager actor
    +		try {
    +			LOG.info("Starting TaskManager actor");
    +			ActorRef taskManagerActor = startTaskManagerComponentsAndActor(
    +				configuration,
    +				resourceID,
    +				taskManagerSystem,
    +				taskManagerHostname,
    +				TASK_MANAGER_NAME,
    +				null,
    +				false,
    +				taskManagerClass);
    +
    +			// start a process reaper that watches the JobManager. If the TaskManager actor dies,
    +			// the process reaper will kill the JVM process (to ensure easy failure detection)
    +			LOG.debug("Starting TaskManager process reaper");
    +
    +			taskManagerSystem.actorOf(
    +				Props.create(ProcessReaper.class, taskManagerActor, LOG, RUNTIME_FAILURE_RETURN_CODE),
    +				"TaskManager_Process_Reaper");
    +
    +			// if desired, start the logging daemon that periodically logs the memory usage information
    +			if (LOG.isInfoEnabled() && configuration.getBoolean(
    +				ConfigConstants.TASK_MANAGER_DEBUG_MEMORY_USAGE_START_LOG_THREAD,
    +				ConfigConstants.DEFAULT_TASK_MANAGER_DEBUG_MEMORY_USAGE_START_LOG_THREAD)) {
    +				LOG.info("Starting periodic memory usage logger");
    +
    +				long interval = configuration.getLong(
    +					ConfigConstants.TASK_MANAGER_DEBUG_MEMORY_USAGE_LOG_INTERVAL_MS,
    +					ConfigConstants.DEFAULT_TASK_MANAGER_DEBUG_MEMORY_USAGE_LOG_INTERVAL_MS);
    +
    +				MemoryLogger logger = new MemoryLogger(LOG, interval, taskManagerSystem);
    +				logger.start();
    +			}
    +
    +			// block until everything is done
    +			taskManagerSystem.awaitTermination();
    +		} catch (Throwable t) {
    +			LOG.error("Error while starting up taskManager", t);
    +			try {
    +				taskManagerSystem.shutdown();
    +			} catch (Throwable tt) {
    +				LOG.warn("Could not cleanly shut down actor system", tt);
    +			}
    +			throw t;
    +		}
    +	}
    +
    +	// --------------------------------------------------------------------------
    +	//  Starting and running the TaskManager
    +	// --------------------------------------------------------------------------
    +
    +	/**
    +	 * @param configuration                 The configuration for the TaskManager.
    +	 * @param resourceID                    The id of the resource which the task manager will run on.
    +	 * @param actorSystem                  The actor system that should run the TaskManager actor.
    +	 * @param taskManagerHostname       The hostname/address that describes the TaskManager's data location.
    +	 * @param taskManagerActorName      Optionally the name of the TaskManager actor. If none is given,
    +	 *                                      the actor will use a random name.
    +	 * @param leaderRetrievalService        Optionally, a leader retrieval service can be provided. If none is given,
    +	 *                                      then a LeaderRetrievalService is constructed from the configuration.
    +	 * @param localTaskManagerCommunication If true, the TaskManager will not initiate the TCP network stack.
    +	 * @param taskManagerClass      The class of the TaskManager actor. May be used to give
    +	 *                                      subclasses that understand additional actor messages.
    +	 * @return An ActorRef to the TaskManager actor.
    +	 * @throws org.apache.flink.configuration.IllegalConfigurationException     Thrown, if the given config contains illegal values.
    +	 * @throws java.io.IOException      Thrown, if any of the I/O components (such as buffer pools,
    +	 *                                       I/O manager, ...) cannot be properly started.
    +	 * @throws java.lang.Exception      Thrown is some other error occurs while parsing the configuration
    +	 *                                      or starting the TaskManager components.
    +	 */
    +	public static ActorRef startTaskManagerComponentsAndActor(
    +		Configuration configuration,
    +		ResourceID resourceID,
    +		ActorSystem actorSystem,
    +		String taskManagerHostname,
    +		String taskManagerActorName,
    +		LeaderRetrievalService leaderRetrievalService,
    +		boolean localTaskManagerCommunication,
    +		Class<? extends TaskManager> taskManagerClass) throws Exception {
    +
    +		Tuple4<TaskManagerConfiguration, NetworkEnvironmentConfiguration, InstanceConnectionInfo, MemoryType> tuple4
    +			= parseTaskManagerConfiguration(configuration, taskManagerHostname, localTaskManagerCommunication);
    +
    +		TaskManagerConfiguration taskManagerConfig = tuple4._1();
    +		NetworkEnvironmentConfiguration netConfig = tuple4._2();
    +		InstanceConnectionInfo connectionInfo = tuple4._3();
    +		MemoryType memType = tuple4._4();
    +
    +		// pre-start checks
    +		checkTempDirs(taskManagerConfig.tmpDirPaths());
    +
    +		ExecutionContext executionContext = ExecutionContexts$.MODULE$.fromExecutor(new ForkJoinPool());
    +
    +		// we start the network first, to make sure it can allocate its buffers first
    +		NetworkEnvironment network = new NetworkEnvironment(executionContext, taskManagerConfig.timeout(), netConfig);
    +
    +		// computing the amount of memory to use depends on how much memory is available
    +		// it strictly needs to happen AFTER the network stack has been initialized
    +
    +		// check if a value has been configured
    +		long configuredMemory = configuration.getLong(ConfigConstants.TASK_MANAGER_MEMORY_SIZE_KEY, -1L);
    +		checkConfigParameter(configuredMemory == -1 || configuredMemory > 0, configuredMemory,
    +			ConfigConstants.TASK_MANAGER_MEMORY_SIZE_KEY,
    +			"MemoryManager needs at least one MB of memory. " +
    +				"If you leave this config parameter empty, the system automatically " +
    +				"pick a fraction of the available memory.");
    +
    +		long memorySize;
    +		boolean preAllocateMemory = configuration.getBoolean(
    +			ConfigConstants.TASK_MANAGER_MEMORY_PRE_ALLOCATE_KEY,
    +			ConfigConstants.DEFAULT_TASK_MANAGER_MEMORY_PRE_ALLOCATE);
    +		if (configuredMemory > 0) {
    +			if (preAllocateMemory) {
    +				LOG.info("Using {} MB for managed memory." , configuredMemory);
    +			} else {
    +				LOG.info("Limiting managed memory to {} MB, memory will be allocated lazily." , configuredMemory);
    +			}
    +			memorySize = configuredMemory << 20; // megabytes to bytes
    +		} else {
    +			float fraction = configuration.getFloat(
    +				ConfigConstants.TASK_MANAGER_MEMORY_FRACTION_KEY,
    +				ConfigConstants.DEFAULT_MEMORY_MANAGER_MEMORY_FRACTION);
    +			checkConfigParameter(fraction > 0.0f && fraction < 1.0f, fraction,
    +				ConfigConstants.TASK_MANAGER_MEMORY_FRACTION_KEY,
    +				"MemoryManager fraction of the free memory must be between 0.0 and 1.0");
    +
    +			if (memType == MemoryType.HEAP) {
    +				long relativeMemSize = (long) (EnvironmentInformation.getSizeOfFreeHeapMemoryWithDefrag() * fraction);
    +				if (preAllocateMemory) {
    +					LOG.info("Using {} of the currently free heap space for managed heap memory ({} MB)." ,
    +						fraction , relativeMemSize >> 20);
    +				} else {
    +					LOG.info("Limiting managed memory to {} of the currently free heap space ({} MB), " +
    +						"memory will be allocated lazily." , fraction , relativeMemSize >> 20);
    +				}
    +				memorySize = relativeMemSize;
    +			} else if (memType == MemoryType.OFF_HEAP) {
    +				// The maximum heap memory has been adjusted according to the fraction
    +				long maxMemory = EnvironmentInformation.getMaxJvmHeapMemory();
    +				long directMemorySize = (long) (maxMemory / (1.0 - fraction) * fraction);
    +				if (preAllocateMemory) {
    +					LOG.info("Using {} of the maximum memory size for managed off-heap memory ({} MB)." ,
    +						fraction, directMemorySize >> 20);
    +				} else {
    +					LOG.info("Limiting managed memory to {} of the maximum memory size ({} MB)," +
    +						" memory will be allocated lazily.", fraction, directMemorySize >> 20);
    +				}
    +				memorySize = directMemorySize;
    +			} else {
    +				throw new RuntimeException("No supported memory type detected.");
    +			}
    +		}
    +
    +		// now start the memory manager
    +		MemoryManager memoryManager;
    +		try {
    +			memoryManager = new MemoryManager(
    +				memorySize,
    +				taskManagerConfig.numberOfSlots(),
    +				netConfig.networkBufferSize(),
    +				memType,
    +				preAllocateMemory);
    +		} catch (OutOfMemoryError e) {
    +			if (memType == MemoryType.HEAP) {
    +				throw new Exception("OutOfMemory error (" + e.getMessage() +
    +					") while allocating the TaskManager heap memory (" + memorySize + " bytes).", e);
    +			} else if (memType == MemoryType.OFF_HEAP) {
    +				throw new Exception("OutOfMemory error (" + e.getMessage() +
    +					") while allocating the TaskManager off-heap memory (" + memorySize +
    +					" bytes).Try increasing the maximum direct memory (-XX:MaxDirectMemorySize)", e);
    +			} else {
    +				throw e;
    +			}
    +		}
    +
    +		// start the I/O manager, it will create some temp directories.
    +		IOManager ioManager = new IOManagerAsync(taskManagerConfig.tmpDirPaths());
    +
    +		if (leaderRetrievalService == null){
    +			leaderRetrievalService = LeaderRetrievalUtils.createLeaderRetrievalService(configuration);
    +		}
    +
    +		// create the actor properties (which define the actor constructor parameters)
    +		Props tmProps = Props.create(
    +			taskManagerClass,
    +			taskManagerConfig,
    +			resourceID,
    +			connectionInfo,
    +			memoryManager,
    +			ioManager,
    +			network,
    +			taskManagerConfig.numberOfSlots(),
    +			leaderRetrievalService);
    +
    +		ActorRef taskManagerActorRef;
    +		if (taskManagerActorName != null && !taskManagerActorName.equals("")) {
    +			taskManagerActorRef = actorSystem.actorOf(tmProps, taskManagerActorName);
    +		} else {
    +			taskManagerActorRef = actorSystem.actorOf(tmProps);
    +		}
    +
    +		return taskManagerActorRef;
    +	}
    +
    +	// --------------------------------------------------------------------------
    +	//  Parsing and checking the TaskManager Configuration
    +	// --------------------------------------------------------------------------
    +
    +	/**
    +	 * Utility method to extract TaskManager config parameters from the configuration and to
    +	 * sanity check them.
    +	 *
    +	 * @param configuration                 The configuration.
    +	 * @param taskManagerHostname           The host name under which the TaskManager communicates.
    +	 * @param localTaskManagerCommunication True, to skip initializing the network stack.
    +	 *                                      Use only in cases where only one task manager runs.
    +	 * @return A tuple (TaskManagerConfiguration, network configuration,
    +	 * InstanceConnectionInfo, JobManager actor Akka URL).
    +	 */
    +	private static Tuple4<TaskManagerConfiguration, NetworkEnvironmentConfiguration, InstanceConnectionInfo, MemoryType>
    +		parseTaskManagerConfiguration(Configuration configuration, String taskManagerHostname, boolean localTaskManagerCommunication)
    +		throws Exception {
    +
    +		// ------- read values from the config and check them ---------
    +		//                      (a lot of them)
    +
    +		// ----> hosts / ports for communication and data exchange
    +
    +		int dataport = configuration.getInteger(ConfigConstants.TASK_MANAGER_DATA_PORT_KEY,
    +			ConfigConstants.DEFAULT_TASK_MANAGER_DATA_PORT);
    +		if (dataport == 0) {
    +			dataport = NetUtils.getAvailablePort();
    +		}
    +		checkConfigParameter(dataport > 0, dataport, ConfigConstants.TASK_MANAGER_DATA_PORT_KEY,
    +			"Leave config parameter empty or use 0 to let the system choose a port automatically.");
    +
    +		InetAddress taskManagerAddress = InetAddress.getByName(taskManagerHostname);
    +		InstanceConnectionInfo connectionInfo = new InstanceConnectionInfo(taskManagerAddress, dataport);
    +
    +		// ----> memory / network stack (shuffles/broadcasts), task slots, temp directories
    +
    +		// we need this because many configs have been written with a "-1" entry
    +		int slots = configuration.getInteger(ConfigConstants.TASK_MANAGER_NUM_TASK_SLOTS, 1);
    +		if (slots == -1) {
    +			slots = 1;
    +		}
    +		checkConfigParameter(slots >= 1, slots, ConfigConstants.TASK_MANAGER_NUM_TASK_SLOTS,
    +			"Number of task slots must be at least one.");
    +
    +		int numNetworkBuffers = configuration.getInteger(
    +			ConfigConstants.TASK_MANAGER_NETWORK_NUM_BUFFERS_KEY,
    +			ConfigConstants.DEFAULT_TASK_MANAGER_NETWORK_NUM_BUFFERS);
    +		checkConfigParameter(numNetworkBuffers > 0, numNetworkBuffers,
    +			ConfigConstants.TASK_MANAGER_NETWORK_NUM_BUFFERS_KEY, "");
    +
    +		int pageSize = configuration.getInteger(
    +			ConfigConstants.TASK_MANAGER_MEMORY_SEGMENT_SIZE_KEY,
    +			ConfigConstants.DEFAULT_TASK_MANAGER_MEMORY_SEGMENT_SIZE);
    +		// check page size of for minimum size
    +		checkConfigParameter(pageSize >= MemoryManager.MIN_PAGE_SIZE, pageSize,
    +			ConfigConstants.TASK_MANAGER_MEMORY_SEGMENT_SIZE_KEY,
    +			"Minimum memory segment size is " + MemoryManager.MIN_PAGE_SIZE);
    +		// check page size for power of two
    +		checkConfigParameter(MathUtils.isPowerOf2(pageSize), pageSize,
    +			ConfigConstants.TASK_MANAGER_MEMORY_SEGMENT_SIZE_KEY,
    +			"Memory segment size must be a power of 2.");
    +
    +		// check whether we use heap or off-heap memory
    +		MemoryType memType;
    +		if (configuration.getBoolean(ConfigConstants.TASK_MANAGER_MEMORY_OFF_HEAP_KEY, false)) {
    +			memType = MemoryType.OFF_HEAP;
    +		} else {
    +			memType = MemoryType.HEAP;
    +		}
    +
    +		// initialize the memory segment factory accordingly
    +		if (memType == MemoryType.HEAP) {
    +			if (!MemorySegmentFactory.isInitialized()) {
    +				MemorySegmentFactory.initializeFactory(HeapMemorySegment.FACTORY);
    +			} else if (MemorySegmentFactory.getFactory() != HeapMemorySegment.FACTORY) {
    +				throw new Exception("Memory type is set to heap memory, but memory segment " +
    +					"factory has been initialized for off-heap memory segments");
    +			}
    +		} else {
    +			if (!MemorySegmentFactory.isInitialized()) {
    +				MemorySegmentFactory.initializeFactory(HybridMemorySegment.FACTORY);
    +			} else if (MemorySegmentFactory.getFactory() != HybridMemorySegment.FACTORY) {
    +				throw new Exception("Memory type is set to off-heap memory, but memory segment " +
    +					"factory has been initialized for heap memory segments");
    +			}
    +		}
    +
    +		String[] tmpDirs = configuration.getString(
    +			ConfigConstants.TASK_MANAGER_TMP_DIR_KEY,
    +			ConfigConstants.DEFAULT_TASK_MANAGER_TMP_PATH).split(",|" + File.pathSeparator);
    +
    +		NettyConfig nettyConfig = null;
    +		if (!localTaskManagerCommunication) {
    +			nettyConfig = new NettyConfig(connectionInfo.address(), connectionInfo.dataPort(), pageSize, slots, configuration);
    +		}
    +
    +		// Default spill I/O mode for intermediate results
    +		String syncOrAsync = configuration.getString(
    +			ConfigConstants.TASK_MANAGER_NETWORK_DEFAULT_IO_MODE,
    +			ConfigConstants.DEFAULT_TASK_MANAGER_NETWORK_DEFAULT_IO_MODE);
    +
    +		IOMode ioMode;
    +		if (syncOrAsync == "async") {
    +			ioMode = IOManager.IOMode.ASYNC;
    +		} else {
    +			ioMode = IOManager.IOMode.SYNC;
    +		}
    +
    +		NetworkEnvironmentConfiguration networkConfig = new NetworkEnvironmentConfiguration(
    +			numNetworkBuffers,
    +			pageSize,
    +			memType,
    +			ioMode,
    +			nettyConfig == null ? Option.<NettyConfig>empty() : new Some<>(nettyConfig),
    +			new Tuple2<>(500, 3000));
    +
    +		// ----> timeouts, library caching, profiling
    +
    +		FiniteDuration timeout;
    +		try {
    +			timeout = AkkaUtils.getTimeout(configuration);
    +		} catch (Exception e) {
    +			throw new IllegalArgumentException(
    +				"Invalid format for '" + ConfigConstants.AKKA_ASK_TIMEOUT +
    +					"'.Use formats like '50 s' or '1 min' to specify the timeout.");
    +		}
    +		LOG.info("Messages between TaskManager and JobManager have a max timeout of " + timeout);
    +
    +		long cleanupInterval = configuration.getLong(
    +			ConfigConstants.LIBRARY_CACHE_MANAGER_CLEANUP_INTERVAL,
    +			ConfigConstants.DEFAULT_LIBRARY_CACHE_MANAGER_CLEANUP_INTERVAL) * 1000;
    +
    +		FiniteDuration finiteRegistrationDuration = null;
    +		try {
    +			Duration maxRegistrationDuration = Duration.create(configuration.getString(
    +				ConfigConstants.TASK_MANAGER_MAX_REGISTRATION_DURATION,
    +				ConfigConstants.DEFAULT_TASK_MANAGER_MAX_REGISTRATION_DURATION));
    +			if (maxRegistrationDuration.isFinite()) {
    +				finiteRegistrationDuration = new FiniteDuration(maxRegistrationDuration.toSeconds(), TimeUnit.SECONDS);
    +			}
    +		} catch (NumberFormatException e) {
    +			throw new IllegalArgumentException("Invalid format for parameter " +
    +				ConfigConstants.TASK_MANAGER_MAX_REGISTRATION_DURATION, e);
    +		}
    +
    +		FiniteDuration initialRegistrationPause;
    +		try {
    +			Duration pause = Duration.create(configuration.getString(
    +				ConfigConstants.TASK_MANAGER_INITIAL_REGISTRATION_PAUSE,
    +				ConfigConstants.DEFAULT_TASK_MANAGER_INITIAL_REGISTRATION_PAUSE));
    +			if (pause.isFinite()) {
    +				initialRegistrationPause = new FiniteDuration(pause.toSeconds(), TimeUnit.SECONDS);
    +			} else {
    +				throw new IllegalArgumentException("The initial registration pause must be finite: " + pause);
    +			}
    +		} catch (NumberFormatException e) {
    +			throw new IllegalArgumentException("Invalid format for parameter " +
    +				ConfigConstants.TASK_MANAGER_INITIAL_REGISTRATION_PAUSE, e);
    +		}
    +
    +		FiniteDuration maxRegistrationPause;
    +		try {
    +			Duration pause = Duration.create(configuration.getString(
    +				ConfigConstants.TASK_MANAGER_MAX_REGISTARTION_PAUSE,
    +				ConfigConstants.DEFAULT_TASK_MANAGER_MAX_REGISTRATION_PAUSE));
    +			if (pause.isFinite()) {
    +				maxRegistrationPause = new FiniteDuration(pause.toSeconds(), TimeUnit.SECONDS);
    +			} else {
    +				throw new IllegalArgumentException("The maximum registration pause must be finite: " + pause);
    +			}
    +		} catch (NumberFormatException e) {
    +			throw new IllegalArgumentException("Invalid format for parameter " +
    +				ConfigConstants.TASK_MANAGER_INITIAL_REGISTRATION_PAUSE, e);
    +		}
    +
    +		FiniteDuration refusedRegistrationPause;
    --- End diff --
    
    `final`


---
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.
---

[GitHub] flink pull request #2400: [FLINK-4363] Implement TaskManager basic startup o...

Posted by mxm <gi...@git.apache.org>.
Github user mxm commented on a diff in the pull request:

    https://github.com/apache/flink/pull/2400#discussion_r75647749
  
    --- Diff: flink-runtime/src/main/java/org/apache/flink/runtime/rpc/taskexecutor/TaskExecutor.java ---
    @@ -35,27 +79,634 @@
      */
     public class TaskExecutor extends RpcEndpoint<TaskExecutorGateway> {
     
    -	/** The unique resource ID of this TaskExecutor */
    +	private static final Logger LOG = LoggerFactory.getLogger(TaskExecutor.class);
    +
    +	/** Return code for critical errors during the runtime */
    +	private static final int RUNTIME_FAILURE_RETURN_CODE = 2;
    +
    +	/** The name of the TaskManager actor */
    +	private static final String TASK_MANAGER_NAME = "taskmanager";
    +
    +	/** The unique resource ID of this TaskManager */
     	private final ResourceID resourceID;
     
     	/** The access to the leader election and metadata storage services */
     	private final HighAvailabilityServices haServices;
     
    -	// --------- resource manager --------
    +	/** The task manager configuration */
    +	private final TaskManagerConfiguration taskManagerConfig;
    +
    +	/** The connection information of the task manager */
    +	private final InstanceConnectionInfo connectionInfo;
    +
    +	/** The I/O manager component in the task manager */
    +	private final IOManager ioManager;
    +
    +	/** The memory manager component in the task manager */
    +	private final MemoryManager memoryManager;
    +
    +	/** The network component in the task manager */
    +	private final NetworkEnvironment networkEnvironment;
    +
    +	/** The number of slots in the task manager, should be 1 for YARN */
    +	private final int numberOfSlots;
     
    +	// --------- resource manager --------
     	private TaskExecutorToResourceManagerConnection resourceManagerConnection;
     
     	// ------------------------------------------------------------------------
     
     	public TaskExecutor(
    +			TaskManagerConfiguration taskManagerConfig,
    +			ResourceID resourceID,
    +			InstanceConnectionInfo connectionInfo,
    +			MemoryManager memoryManager,
    +			IOManager ioManager,
    +			NetworkEnvironment networkEnvironment,
    +			int numberOfSlots,
     			RpcService rpcService,
    -			HighAvailabilityServices haServices,
    -			ResourceID resourceID) {
    +			HighAvailabilityServices haServices) {
     
     		super(rpcService);
     
    -		this.haServices = checkNotNull(haServices);
    +		this.taskManagerConfig = checkNotNull(taskManagerConfig);
     		this.resourceID = checkNotNull(resourceID);
    +		this.connectionInfo = checkNotNull(connectionInfo);
    +		this.memoryManager = checkNotNull(memoryManager);
    +		this.ioManager = checkNotNull(ioManager);
    +		this.networkEnvironment = checkNotNull(networkEnvironment);
    +		this.numberOfSlots = checkNotNull(numberOfSlots);
    +		this.haServices = checkNotNull(haServices);
    +	}
    +
    +	/**
    +	 * Starts and runs the TaskManager.
    +	 * <p/>
    +	 * This method first tries to select the network interface to use for the TaskManager
    +	 * communication. The network interface is used both for the actor communication
    +	 * (coordination) as well as for the data exchange between task managers. Unless
    +	 * the hostname/interface is explicitly configured in the configuration, this
    +	 * method will try out various interfaces and methods to connect to the JobManager
    +	 * and select the one where the connection attempt is successful.
    +	 * <p/>
    +	 * After selecting the network interface, this method brings up an actor system
    +	 * for the TaskManager and its actors, starts the TaskManager's services
    +	 * (library cache, shuffle network stack, ...), and starts the TaskManager itself.
    +	 *
    +	 * @param configuration    The configuration for the TaskManager.
    +	 * @param taskManagerClass The actor class to instantiate.
    +	 *                         Allows to use TaskManager subclasses for example for YARN.
    +	 */
    +	public static void selectNetworkInterfaceAndRunTaskManager(
    +		Configuration configuration,
    +		ResourceID resourceID,
    +		Class<? extends TaskManager> taskManagerClass) throws Exception {
    +
    +		Tuple2<String, Integer> tuple2 = selectNetworkInterfaceAndPort(configuration);
    +
    +		runTaskManager(tuple2._1(), resourceID, tuple2._2(), configuration, taskManagerClass);
    +	}
    +
    +	private static Tuple2<String, Integer> selectNetworkInterfaceAndPort(Configuration configuration)
    +		throws Exception {
    +		String taskManagerHostname = configuration.getString(ConfigConstants.TASK_MANAGER_HOSTNAME_KEY, null);
    +		if (taskManagerHostname != null) {
    +			LOG.info("Using configured hostname/address for TaskManager: " + taskManagerHostname);
    +		} else {
    +			LeaderRetrievalService leaderRetrievalService = LeaderRetrievalUtils.createLeaderRetrievalService(configuration);
    +			FiniteDuration lookupTimeout = AkkaUtils.getLookupTimeout(configuration);
    +
    +			InetAddress taskManagerAddress = LeaderRetrievalUtils.findConnectingAddress(leaderRetrievalService, lookupTimeout);
    +			taskManagerHostname = taskManagerAddress.getHostName();
    +			LOG.info("TaskManager will use hostname/address '{}' ({}) for communication.",
    +				taskManagerHostname, taskManagerAddress.getHostAddress());
    +		}
    +
    +		// if no task manager port has been configured, use 0 (system will pick any free port)
    +		int actorSystemPort = configuration.getInteger(ConfigConstants.TASK_MANAGER_IPC_PORT_KEY, 0);
    +		if (actorSystemPort < 0 || actorSystemPort > 65535) {
    +			throw new IllegalConfigurationException("Invalid value for '" +
    +				ConfigConstants.TASK_MANAGER_IPC_PORT_KEY +
    +				"' (port for the TaskManager actor system) : " + actorSystemPort +
    +				" - Leave config parameter empty or use 0 to let the system choose a port automatically.");
    +		}
    +
    +		return new Tuple2<>(taskManagerHostname, actorSystemPort);
    +	}
    +
    +	/**
    +	 * Starts and runs the TaskManager. Brings up an actor system for the TaskManager and its
    +	 * actors, starts the TaskManager's services (library cache, shuffle network stack, ...),
    +	 * and starts the TaskManager itself.
    +	 * <p/>
    +	 * This method will also spawn a process reaper for the TaskManager (kill the process if
    +	 * the actor fails) and optionally start the JVM memory logging thread.
    +	 *
    +	 * @param taskManagerHostname The hostname/address of the interface where the actor system
    +	 *                            will communicate.
    +	 * @param resourceID          The id of the resource which the task manager will run on.
    +	 * @param actorSystemPort     The port at which the actor system will communicate.
    +	 * @param configuration       The configuration for the TaskManager.
    +	 * @param taskManagerClass    The actor class to instantiate. Allows the use of TaskManager
    +	 *                            subclasses for example for YARN.
    +	 */
    +	private static void runTaskManager(
    +		String taskManagerHostname,
    +		ResourceID resourceID,
    +		int actorSystemPort,
    +		Configuration configuration,
    +		Class<? extends TaskManager> taskManagerClass) throws Exception {
    +
    +		LOG.info("Starting TaskManager");
    +
    +		// Bring up the TaskManager actor system first, bind it to the given address.
    +
    +		LOG.info("Starting TaskManager actor system at " +
    +			NetUtils.hostAndPortToUrlString(taskManagerHostname, actorSystemPort));
    +
    +		ActorSystem taskManagerSystem;
    +		try {
    +			Tuple2<String, Object> address = new Tuple2<String, Object>(taskManagerHostname, actorSystemPort);
    +			Config akkaConfig = AkkaUtils.getAkkaConfig(configuration, new Some<>(address));
    +			LOG.debug("Using akka configuration\n " + akkaConfig);
    +			taskManagerSystem = AkkaUtils.createActorSystem(akkaConfig);
    +		} catch (Throwable t) {
    +			if (t instanceof org.jboss.netty.channel.ChannelException) {
    +				Throwable cause = t.getCause();
    +				if (cause != null && t.getCause() instanceof java.net.BindException) {
    +					String address = NetUtils.hostAndPortToUrlString(taskManagerHostname, actorSystemPort);
    +					throw new IOException("Unable to bind TaskManager actor system to address " +
    +						address + " - " + cause.getMessage(), t);
    +				}
    +			}
    +			throw new Exception("Could not create TaskManager actor system", t);
    +		}
    +
    +		// start all the TaskManager services (network stack,  library cache, ...)
    +		// and the TaskManager actor
    +		try {
    +			LOG.info("Starting TaskManager actor");
    +			ActorRef taskManagerActor = startTaskManagerComponentsAndActor(
    +				configuration,
    +				resourceID,
    +				taskManagerSystem,
    +				taskManagerHostname,
    +				TASK_MANAGER_NAME,
    +				null,
    +				false,
    +				taskManagerClass);
    +
    +			// start a process reaper that watches the JobManager. If the TaskManager actor dies,
    +			// the process reaper will kill the JVM process (to ensure easy failure detection)
    +			LOG.debug("Starting TaskManager process reaper");
    +
    +			taskManagerSystem.actorOf(
    +				Props.create(ProcessReaper.class, taskManagerActor, LOG, RUNTIME_FAILURE_RETURN_CODE),
    +				"TaskManager_Process_Reaper");
    +
    +			// if desired, start the logging daemon that periodically logs the memory usage information
    +			if (LOG.isInfoEnabled() && configuration.getBoolean(
    +				ConfigConstants.TASK_MANAGER_DEBUG_MEMORY_USAGE_START_LOG_THREAD,
    +				ConfigConstants.DEFAULT_TASK_MANAGER_DEBUG_MEMORY_USAGE_START_LOG_THREAD)) {
    +				LOG.info("Starting periodic memory usage logger");
    +
    +				long interval = configuration.getLong(
    +					ConfigConstants.TASK_MANAGER_DEBUG_MEMORY_USAGE_LOG_INTERVAL_MS,
    +					ConfigConstants.DEFAULT_TASK_MANAGER_DEBUG_MEMORY_USAGE_LOG_INTERVAL_MS);
    +
    +				MemoryLogger logger = new MemoryLogger(LOG, interval, taskManagerSystem);
    +				logger.start();
    +			}
    +
    +			// block until everything is done
    +			taskManagerSystem.awaitTermination();
    +		} catch (Throwable t) {
    +			LOG.error("Error while starting up taskManager", t);
    +			try {
    +				taskManagerSystem.shutdown();
    +			} catch (Throwable tt) {
    +				LOG.warn("Could not cleanly shut down actor system", tt);
    +			}
    +			throw t;
    +		}
    +	}
    +
    +	// --------------------------------------------------------------------------
    +	//  Starting and running the TaskManager
    +	// --------------------------------------------------------------------------
    +
    +	/**
    +	 * @param configuration                 The configuration for the TaskManager.
    +	 * @param resourceID                    The id of the resource which the task manager will run on.
    +	 * @param actorSystem                  The actor system that should run the TaskManager actor.
    +	 * @param taskManagerHostname       The hostname/address that describes the TaskManager's data location.
    +	 * @param taskManagerActorName      Optionally the name of the TaskManager actor. If none is given,
    +	 *                                      the actor will use a random name.
    +	 * @param leaderRetrievalService        Optionally, a leader retrieval service can be provided. If none is given,
    +	 *                                      then a LeaderRetrievalService is constructed from the configuration.
    +	 * @param localTaskManagerCommunication If true, the TaskManager will not initiate the TCP network stack.
    +	 * @param taskManagerClass      The class of the TaskManager actor. May be used to give
    +	 *                                      subclasses that understand additional actor messages.
    +	 * @return An ActorRef to the TaskManager actor.
    +	 * @throws org.apache.flink.configuration.IllegalConfigurationException     Thrown, if the given config contains illegal values.
    +	 * @throws java.io.IOException      Thrown, if any of the I/O components (such as buffer pools,
    +	 *                                       I/O manager, ...) cannot be properly started.
    +	 * @throws java.lang.Exception      Thrown is some other error occurs while parsing the configuration
    +	 *                                      or starting the TaskManager components.
    +	 */
    +	public static ActorRef startTaskManagerComponentsAndActor(
    +		Configuration configuration,
    +		ResourceID resourceID,
    +		ActorSystem actorSystem,
    +		String taskManagerHostname,
    +		String taskManagerActorName,
    +		LeaderRetrievalService leaderRetrievalService,
    +		boolean localTaskManagerCommunication,
    +		Class<? extends TaskManager> taskManagerClass) throws Exception {
    +
    +		Tuple4<TaskManagerConfiguration, NetworkEnvironmentConfiguration, InstanceConnectionInfo, MemoryType> tuple4
    +			= parseTaskManagerConfiguration(configuration, taskManagerHostname, localTaskManagerCommunication);
    +
    +		TaskManagerConfiguration taskManagerConfig = tuple4._1();
    +		NetworkEnvironmentConfiguration netConfig = tuple4._2();
    +		InstanceConnectionInfo connectionInfo = tuple4._3();
    +		MemoryType memType = tuple4._4();
    +
    +		// pre-start checks
    +		checkTempDirs(taskManagerConfig.tmpDirPaths());
    +
    +		ExecutionContext executionContext = ExecutionContexts$.MODULE$.fromExecutor(new ForkJoinPool());
    +
    +		// we start the network first, to make sure it can allocate its buffers first
    +		NetworkEnvironment network = new NetworkEnvironment(executionContext, taskManagerConfig.timeout(), netConfig);
    +
    +		// computing the amount of memory to use depends on how much memory is available
    +		// it strictly needs to happen AFTER the network stack has been initialized
    +
    +		// check if a value has been configured
    +		long configuredMemory = configuration.getLong(ConfigConstants.TASK_MANAGER_MEMORY_SIZE_KEY, -1L);
    +		checkConfigParameter(configuredMemory == -1 || configuredMemory > 0, configuredMemory,
    +			ConfigConstants.TASK_MANAGER_MEMORY_SIZE_KEY,
    +			"MemoryManager needs at least one MB of memory. " +
    +				"If you leave this config parameter empty, the system automatically " +
    +				"pick a fraction of the available memory.");
    +
    +		long memorySize;
    +		boolean preAllocateMemory = configuration.getBoolean(
    +			ConfigConstants.TASK_MANAGER_MEMORY_PRE_ALLOCATE_KEY,
    +			ConfigConstants.DEFAULT_TASK_MANAGER_MEMORY_PRE_ALLOCATE);
    +		if (configuredMemory > 0) {
    +			if (preAllocateMemory) {
    +				LOG.info("Using {} MB for managed memory." , configuredMemory);
    +			} else {
    +				LOG.info("Limiting managed memory to {} MB, memory will be allocated lazily." , configuredMemory);
    +			}
    +			memorySize = configuredMemory << 20; // megabytes to bytes
    +		} else {
    +			float fraction = configuration.getFloat(
    +				ConfigConstants.TASK_MANAGER_MEMORY_FRACTION_KEY,
    +				ConfigConstants.DEFAULT_MEMORY_MANAGER_MEMORY_FRACTION);
    +			checkConfigParameter(fraction > 0.0f && fraction < 1.0f, fraction,
    +				ConfigConstants.TASK_MANAGER_MEMORY_FRACTION_KEY,
    +				"MemoryManager fraction of the free memory must be between 0.0 and 1.0");
    +
    +			if (memType == MemoryType.HEAP) {
    +				long relativeMemSize = (long) (EnvironmentInformation.getSizeOfFreeHeapMemoryWithDefrag() * fraction);
    +				if (preAllocateMemory) {
    +					LOG.info("Using {} of the currently free heap space for managed heap memory ({} MB)." ,
    +						fraction , relativeMemSize >> 20);
    +				} else {
    +					LOG.info("Limiting managed memory to {} of the currently free heap space ({} MB), " +
    +						"memory will be allocated lazily." , fraction , relativeMemSize >> 20);
    +				}
    +				memorySize = relativeMemSize;
    +			} else if (memType == MemoryType.OFF_HEAP) {
    +				// The maximum heap memory has been adjusted according to the fraction
    +				long maxMemory = EnvironmentInformation.getMaxJvmHeapMemory();
    +				long directMemorySize = (long) (maxMemory / (1.0 - fraction) * fraction);
    +				if (preAllocateMemory) {
    +					LOG.info("Using {} of the maximum memory size for managed off-heap memory ({} MB)." ,
    +						fraction, directMemorySize >> 20);
    +				} else {
    +					LOG.info("Limiting managed memory to {} of the maximum memory size ({} MB)," +
    +						" memory will be allocated lazily.", fraction, directMemorySize >> 20);
    +				}
    +				memorySize = directMemorySize;
    +			} else {
    +				throw new RuntimeException("No supported memory type detected.");
    +			}
    +		}
    +
    +		// now start the memory manager
    +		MemoryManager memoryManager;
    +		try {
    +			memoryManager = new MemoryManager(
    +				memorySize,
    +				taskManagerConfig.numberOfSlots(),
    +				netConfig.networkBufferSize(),
    +				memType,
    +				preAllocateMemory);
    +		} catch (OutOfMemoryError e) {
    +			if (memType == MemoryType.HEAP) {
    +				throw new Exception("OutOfMemory error (" + e.getMessage() +
    +					") while allocating the TaskManager heap memory (" + memorySize + " bytes).", e);
    +			} else if (memType == MemoryType.OFF_HEAP) {
    +				throw new Exception("OutOfMemory error (" + e.getMessage() +
    +					") while allocating the TaskManager off-heap memory (" + memorySize +
    +					" bytes).Try increasing the maximum direct memory (-XX:MaxDirectMemorySize)", e);
    +			} else {
    +				throw e;
    +			}
    +		}
    +
    +		// start the I/O manager, it will create some temp directories.
    +		IOManager ioManager = new IOManagerAsync(taskManagerConfig.tmpDirPaths());
    +
    +		if (leaderRetrievalService == null){
    +			leaderRetrievalService = LeaderRetrievalUtils.createLeaderRetrievalService(configuration);
    +		}
    +
    +		// create the actor properties (which define the actor constructor parameters)
    +		Props tmProps = Props.create(
    +			taskManagerClass,
    +			taskManagerConfig,
    +			resourceID,
    +			connectionInfo,
    +			memoryManager,
    +			ioManager,
    +			network,
    +			taskManagerConfig.numberOfSlots(),
    +			leaderRetrievalService);
    +
    +		ActorRef taskManagerActorRef;
    +		if (taskManagerActorName != null && !taskManagerActorName.equals("")) {
    +			taskManagerActorRef = actorSystem.actorOf(tmProps, taskManagerActorName);
    +		} else {
    +			taskManagerActorRef = actorSystem.actorOf(tmProps);
    +		}
    +
    +		return taskManagerActorRef;
    +	}
    +
    +	// --------------------------------------------------------------------------
    +	//  Parsing and checking the TaskManager Configuration
    +	// --------------------------------------------------------------------------
    +
    +	/**
    +	 * Utility method to extract TaskManager config parameters from the configuration and to
    +	 * sanity check them.
    +	 *
    +	 * @param configuration                 The configuration.
    +	 * @param taskManagerHostname           The host name under which the TaskManager communicates.
    +	 * @param localTaskManagerCommunication True, to skip initializing the network stack.
    +	 *                                      Use only in cases where only one task manager runs.
    +	 * @return A tuple (TaskManagerConfiguration, network configuration,
    +	 * InstanceConnectionInfo, JobManager actor Akka URL).
    +	 */
    +	private static Tuple4<TaskManagerConfiguration, NetworkEnvironmentConfiguration, InstanceConnectionInfo, MemoryType>
    --- End diff --
    
    I would create a static class to hold the results instead of using Scala tuples.


---
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.
---

[GitHub] flink pull request #2400: [FLINK-4363] Implement TaskManager basic startup o...

Posted by mxm <gi...@git.apache.org>.
Github user mxm commented on a diff in the pull request:

    https://github.com/apache/flink/pull/2400#discussion_r75644911
  
    --- Diff: flink-runtime/src/main/java/org/apache/flink/runtime/rpc/taskexecutor/TaskExecutor.java ---
    @@ -35,27 +79,634 @@
      */
     public class TaskExecutor extends RpcEndpoint<TaskExecutorGateway> {
     
    -	/** The unique resource ID of this TaskExecutor */
    +	private static final Logger LOG = LoggerFactory.getLogger(TaskExecutor.class);
    +
    +	/** Return code for critical errors during the runtime */
    +	private static final int RUNTIME_FAILURE_RETURN_CODE = 2;
    +
    +	/** The name of the TaskManager actor */
    +	private static final String TASK_MANAGER_NAME = "taskmanager";
    +
    +	/** The unique resource ID of this TaskManager */
     	private final ResourceID resourceID;
     
     	/** The access to the leader election and metadata storage services */
     	private final HighAvailabilityServices haServices;
     
    -	// --------- resource manager --------
    +	/** The task manager configuration */
    +	private final TaskManagerConfiguration taskManagerConfig;
    +
    +	/** The connection information of the task manager */
    +	private final InstanceConnectionInfo connectionInfo;
    +
    +	/** The I/O manager component in the task manager */
    +	private final IOManager ioManager;
    +
    +	/** The memory manager component in the task manager */
    +	private final MemoryManager memoryManager;
    +
    +	/** The network component in the task manager */
    +	private final NetworkEnvironment networkEnvironment;
    +
    +	/** The number of slots in the task manager, should be 1 for YARN */
    +	private final int numberOfSlots;
     
    +	// --------- resource manager --------
     	private TaskExecutorToResourceManagerConnection resourceManagerConnection;
     
     	// ------------------------------------------------------------------------
     
     	public TaskExecutor(
    +			TaskManagerConfiguration taskManagerConfig,
    +			ResourceID resourceID,
    +			InstanceConnectionInfo connectionInfo,
    +			MemoryManager memoryManager,
    +			IOManager ioManager,
    +			NetworkEnvironment networkEnvironment,
    +			int numberOfSlots,
     			RpcService rpcService,
    -			HighAvailabilityServices haServices,
    -			ResourceID resourceID) {
    +			HighAvailabilityServices haServices) {
     
     		super(rpcService);
     
    -		this.haServices = checkNotNull(haServices);
    +		this.taskManagerConfig = checkNotNull(taskManagerConfig);
     		this.resourceID = checkNotNull(resourceID);
    +		this.connectionInfo = checkNotNull(connectionInfo);
    +		this.memoryManager = checkNotNull(memoryManager);
    +		this.ioManager = checkNotNull(ioManager);
    +		this.networkEnvironment = checkNotNull(networkEnvironment);
    +		this.numberOfSlots = checkNotNull(numberOfSlots);
    +		this.haServices = checkNotNull(haServices);
    +	}
    +
    +	/**
    +	 * Starts and runs the TaskManager.
    +	 * <p/>
    +	 * This method first tries to select the network interface to use for the TaskManager
    +	 * communication. The network interface is used both for the actor communication
    +	 * (coordination) as well as for the data exchange between task managers. Unless
    +	 * the hostname/interface is explicitly configured in the configuration, this
    +	 * method will try out various interfaces and methods to connect to the JobManager
    +	 * and select the one where the connection attempt is successful.
    +	 * <p/>
    +	 * After selecting the network interface, this method brings up an actor system
    +	 * for the TaskManager and its actors, starts the TaskManager's services
    +	 * (library cache, shuffle network stack, ...), and starts the TaskManager itself.
    +	 *
    +	 * @param configuration    The configuration for the TaskManager.
    +	 * @param taskManagerClass The actor class to instantiate.
    +	 *                         Allows to use TaskManager subclasses for example for YARN.
    +	 */
    +	public static void selectNetworkInterfaceAndRunTaskManager(
    +		Configuration configuration,
    +		ResourceID resourceID,
    +		Class<? extends TaskManager> taskManagerClass) throws Exception {
    +
    +		Tuple2<String, Integer> tuple2 = selectNetworkInterfaceAndPort(configuration);
    +
    +		runTaskManager(tuple2._1(), resourceID, tuple2._2(), configuration, taskManagerClass);
    +	}
    +
    +	private static Tuple2<String, Integer> selectNetworkInterfaceAndPort(Configuration configuration)
    +		throws Exception {
    +		String taskManagerHostname = configuration.getString(ConfigConstants.TASK_MANAGER_HOSTNAME_KEY, null);
    +		if (taskManagerHostname != null) {
    +			LOG.info("Using configured hostname/address for TaskManager: " + taskManagerHostname);
    +		} else {
    +			LeaderRetrievalService leaderRetrievalService = LeaderRetrievalUtils.createLeaderRetrievalService(configuration);
    +			FiniteDuration lookupTimeout = AkkaUtils.getLookupTimeout(configuration);
    +
    +			InetAddress taskManagerAddress = LeaderRetrievalUtils.findConnectingAddress(leaderRetrievalService, lookupTimeout);
    +			taskManagerHostname = taskManagerAddress.getHostName();
    +			LOG.info("TaskManager will use hostname/address '{}' ({}) for communication.",
    +				taskManagerHostname, taskManagerAddress.getHostAddress());
    +		}
    +
    +		// if no task manager port has been configured, use 0 (system will pick any free port)
    +		int actorSystemPort = configuration.getInteger(ConfigConstants.TASK_MANAGER_IPC_PORT_KEY, 0);
    +		if (actorSystemPort < 0 || actorSystemPort > 65535) {
    +			throw new IllegalConfigurationException("Invalid value for '" +
    +				ConfigConstants.TASK_MANAGER_IPC_PORT_KEY +
    +				"' (port for the TaskManager actor system) : " + actorSystemPort +
    +				" - Leave config parameter empty or use 0 to let the system choose a port automatically.");
    +		}
    +
    +		return new Tuple2<>(taskManagerHostname, actorSystemPort);
    +	}
    +
    +	/**
    +	 * Starts and runs the TaskManager. Brings up an actor system for the TaskManager and its
    +	 * actors, starts the TaskManager's services (library cache, shuffle network stack, ...),
    +	 * and starts the TaskManager itself.
    +	 * <p/>
    +	 * This method will also spawn a process reaper for the TaskManager (kill the process if
    +	 * the actor fails) and optionally start the JVM memory logging thread.
    +	 *
    +	 * @param taskManagerHostname The hostname/address of the interface where the actor system
    +	 *                            will communicate.
    +	 * @param resourceID          The id of the resource which the task manager will run on.
    +	 * @param actorSystemPort     The port at which the actor system will communicate.
    +	 * @param configuration       The configuration for the TaskManager.
    +	 * @param taskManagerClass    The actor class to instantiate. Allows the use of TaskManager
    +	 *                            subclasses for example for YARN.
    +	 */
    +	private static void runTaskManager(
    +		String taskManagerHostname,
    +		ResourceID resourceID,
    +		int actorSystemPort,
    +		Configuration configuration,
    +		Class<? extends TaskManager> taskManagerClass) throws Exception {
    +
    +		LOG.info("Starting TaskManager");
    +
    +		// Bring up the TaskManager actor system first, bind it to the given address.
    +
    +		LOG.info("Starting TaskManager actor system at " +
    +			NetUtils.hostAndPortToUrlString(taskManagerHostname, actorSystemPort));
    +
    +		ActorSystem taskManagerSystem;
    +		try {
    +			Tuple2<String, Object> address = new Tuple2<String, Object>(taskManagerHostname, actorSystemPort);
    +			Config akkaConfig = AkkaUtils.getAkkaConfig(configuration, new Some<>(address));
    +			LOG.debug("Using akka configuration\n " + akkaConfig);
    +			taskManagerSystem = AkkaUtils.createActorSystem(akkaConfig);
    +		} catch (Throwable t) {
    +			if (t instanceof org.jboss.netty.channel.ChannelException) {
    +				Throwable cause = t.getCause();
    +				if (cause != null && t.getCause() instanceof java.net.BindException) {
    +					String address = NetUtils.hostAndPortToUrlString(taskManagerHostname, actorSystemPort);
    +					throw new IOException("Unable to bind TaskManager actor system to address " +
    +						address + " - " + cause.getMessage(), t);
    +				}
    +			}
    +			throw new Exception("Could not create TaskManager actor system", t);
    +		}
    +
    +		// start all the TaskManager services (network stack,  library cache, ...)
    +		// and the TaskManager actor
    +		try {
    +			LOG.info("Starting TaskManager actor");
    +			ActorRef taskManagerActor = startTaskManagerComponentsAndActor(
    +				configuration,
    +				resourceID,
    +				taskManagerSystem,
    +				taskManagerHostname,
    +				TASK_MANAGER_NAME,
    +				null,
    +				false,
    +				taskManagerClass);
    +
    +			// start a process reaper that watches the JobManager. If the TaskManager actor dies,
    +			// the process reaper will kill the JVM process (to ensure easy failure detection)
    +			LOG.debug("Starting TaskManager process reaper");
    +
    +			taskManagerSystem.actorOf(
    +				Props.create(ProcessReaper.class, taskManagerActor, LOG, RUNTIME_FAILURE_RETURN_CODE),
    +				"TaskManager_Process_Reaper");
    +
    +			// if desired, start the logging daemon that periodically logs the memory usage information
    +			if (LOG.isInfoEnabled() && configuration.getBoolean(
    +				ConfigConstants.TASK_MANAGER_DEBUG_MEMORY_USAGE_START_LOG_THREAD,
    +				ConfigConstants.DEFAULT_TASK_MANAGER_DEBUG_MEMORY_USAGE_START_LOG_THREAD)) {
    +				LOG.info("Starting periodic memory usage logger");
    +
    +				long interval = configuration.getLong(
    +					ConfigConstants.TASK_MANAGER_DEBUG_MEMORY_USAGE_LOG_INTERVAL_MS,
    +					ConfigConstants.DEFAULT_TASK_MANAGER_DEBUG_MEMORY_USAGE_LOG_INTERVAL_MS);
    +
    +				MemoryLogger logger = new MemoryLogger(LOG, interval, taskManagerSystem);
    +				logger.start();
    +			}
    +
    +			// block until everything is done
    +			taskManagerSystem.awaitTermination();
    +		} catch (Throwable t) {
    +			LOG.error("Error while starting up taskManager", t);
    +			try {
    +				taskManagerSystem.shutdown();
    +			} catch (Throwable tt) {
    +				LOG.warn("Could not cleanly shut down actor system", tt);
    +			}
    +			throw t;
    +		}
    +	}
    +
    +	// --------------------------------------------------------------------------
    +	//  Starting and running the TaskManager
    +	// --------------------------------------------------------------------------
    +
    +	/**
    +	 * @param configuration                 The configuration for the TaskManager.
    +	 * @param resourceID                    The id of the resource which the task manager will run on.
    +	 * @param actorSystem                  The actor system that should run the TaskManager actor.
    +	 * @param taskManagerHostname       The hostname/address that describes the TaskManager's data location.
    +	 * @param taskManagerActorName      Optionally the name of the TaskManager actor. If none is given,
    +	 *                                      the actor will use a random name.
    +	 * @param leaderRetrievalService        Optionally, a leader retrieval service can be provided. If none is given,
    +	 *                                      then a LeaderRetrievalService is constructed from the configuration.
    +	 * @param localTaskManagerCommunication If true, the TaskManager will not initiate the TCP network stack.
    +	 * @param taskManagerClass      The class of the TaskManager actor. May be used to give
    +	 *                                      subclasses that understand additional actor messages.
    +	 * @return An ActorRef to the TaskManager actor.
    +	 * @throws org.apache.flink.configuration.IllegalConfigurationException     Thrown, if the given config contains illegal values.
    +	 * @throws java.io.IOException      Thrown, if any of the I/O components (such as buffer pools,
    +	 *                                       I/O manager, ...) cannot be properly started.
    +	 * @throws java.lang.Exception      Thrown is some other error occurs while parsing the configuration
    +	 *                                      or starting the TaskManager components.
    +	 */
    +	public static ActorRef startTaskManagerComponentsAndActor(
    +		Configuration configuration,
    +		ResourceID resourceID,
    +		ActorSystem actorSystem,
    +		String taskManagerHostname,
    +		String taskManagerActorName,
    +		LeaderRetrievalService leaderRetrievalService,
    +		boolean localTaskManagerCommunication,
    +		Class<? extends TaskManager> taskManagerClass) throws Exception {
    +
    +		Tuple4<TaskManagerConfiguration, NetworkEnvironmentConfiguration, InstanceConnectionInfo, MemoryType> tuple4
    +			= parseTaskManagerConfiguration(configuration, taskManagerHostname, localTaskManagerCommunication);
    +
    +		TaskManagerConfiguration taskManagerConfig = tuple4._1();
    +		NetworkEnvironmentConfiguration netConfig = tuple4._2();
    +		InstanceConnectionInfo connectionInfo = tuple4._3();
    +		MemoryType memType = tuple4._4();
    +
    +		// pre-start checks
    +		checkTempDirs(taskManagerConfig.tmpDirPaths());
    +
    +		ExecutionContext executionContext = ExecutionContexts$.MODULE$.fromExecutor(new ForkJoinPool());
    +
    +		// we start the network first, to make sure it can allocate its buffers first
    +		NetworkEnvironment network = new NetworkEnvironment(executionContext, taskManagerConfig.timeout(), netConfig);
    +
    +		// computing the amount of memory to use depends on how much memory is available
    +		// it strictly needs to happen AFTER the network stack has been initialized
    +
    +		// check if a value has been configured
    +		long configuredMemory = configuration.getLong(ConfigConstants.TASK_MANAGER_MEMORY_SIZE_KEY, -1L);
    +		checkConfigParameter(configuredMemory == -1 || configuredMemory > 0, configuredMemory,
    +			ConfigConstants.TASK_MANAGER_MEMORY_SIZE_KEY,
    +			"MemoryManager needs at least one MB of memory. " +
    +				"If you leave this config parameter empty, the system automatically " +
    +				"pick a fraction of the available memory.");
    +
    +		long memorySize;
    +		boolean preAllocateMemory = configuration.getBoolean(
    +			ConfigConstants.TASK_MANAGER_MEMORY_PRE_ALLOCATE_KEY,
    +			ConfigConstants.DEFAULT_TASK_MANAGER_MEMORY_PRE_ALLOCATE);
    +		if (configuredMemory > 0) {
    +			if (preAllocateMemory) {
    +				LOG.info("Using {} MB for managed memory." , configuredMemory);
    +			} else {
    +				LOG.info("Limiting managed memory to {} MB, memory will be allocated lazily." , configuredMemory);
    +			}
    +			memorySize = configuredMemory << 20; // megabytes to bytes
    +		} else {
    +			float fraction = configuration.getFloat(
    +				ConfigConstants.TASK_MANAGER_MEMORY_FRACTION_KEY,
    +				ConfigConstants.DEFAULT_MEMORY_MANAGER_MEMORY_FRACTION);
    +			checkConfigParameter(fraction > 0.0f && fraction < 1.0f, fraction,
    +				ConfigConstants.TASK_MANAGER_MEMORY_FRACTION_KEY,
    +				"MemoryManager fraction of the free memory must be between 0.0 and 1.0");
    +
    +			if (memType == MemoryType.HEAP) {
    +				long relativeMemSize = (long) (EnvironmentInformation.getSizeOfFreeHeapMemoryWithDefrag() * fraction);
    +				if (preAllocateMemory) {
    +					LOG.info("Using {} of the currently free heap space for managed heap memory ({} MB)." ,
    +						fraction , relativeMemSize >> 20);
    +				} else {
    +					LOG.info("Limiting managed memory to {} of the currently free heap space ({} MB), " +
    +						"memory will be allocated lazily." , fraction , relativeMemSize >> 20);
    +				}
    +				memorySize = relativeMemSize;
    +			} else if (memType == MemoryType.OFF_HEAP) {
    +				// The maximum heap memory has been adjusted according to the fraction
    +				long maxMemory = EnvironmentInformation.getMaxJvmHeapMemory();
    +				long directMemorySize = (long) (maxMemory / (1.0 - fraction) * fraction);
    +				if (preAllocateMemory) {
    +					LOG.info("Using {} of the maximum memory size for managed off-heap memory ({} MB)." ,
    +						fraction, directMemorySize >> 20);
    +				} else {
    +					LOG.info("Limiting managed memory to {} of the maximum memory size ({} MB)," +
    +						" memory will be allocated lazily.", fraction, directMemorySize >> 20);
    +				}
    +				memorySize = directMemorySize;
    +			} else {
    +				throw new RuntimeException("No supported memory type detected.");
    +			}
    +		}
    +
    +		// now start the memory manager
    +		MemoryManager memoryManager;
    +		try {
    +			memoryManager = new MemoryManager(
    +				memorySize,
    +				taskManagerConfig.numberOfSlots(),
    +				netConfig.networkBufferSize(),
    +				memType,
    +				preAllocateMemory);
    +		} catch (OutOfMemoryError e) {
    +			if (memType == MemoryType.HEAP) {
    +				throw new Exception("OutOfMemory error (" + e.getMessage() +
    +					") while allocating the TaskManager heap memory (" + memorySize + " bytes).", e);
    +			} else if (memType == MemoryType.OFF_HEAP) {
    +				throw new Exception("OutOfMemory error (" + e.getMessage() +
    +					") while allocating the TaskManager off-heap memory (" + memorySize +
    +					" bytes).Try increasing the maximum direct memory (-XX:MaxDirectMemorySize)", e);
    +			} else {
    +				throw e;
    +			}
    +		}
    +
    +		// start the I/O manager, it will create some temp directories.
    +		IOManager ioManager = new IOManagerAsync(taskManagerConfig.tmpDirPaths());
    +
    +		if (leaderRetrievalService == null){
    +			leaderRetrievalService = LeaderRetrievalUtils.createLeaderRetrievalService(configuration);
    +		}
    +
    +		// create the actor properties (which define the actor constructor parameters)
    +		Props tmProps = Props.create(
    +			taskManagerClass,
    +			taskManagerConfig,
    +			resourceID,
    +			connectionInfo,
    +			memoryManager,
    +			ioManager,
    +			network,
    +			taskManagerConfig.numberOfSlots(),
    +			leaderRetrievalService);
    +
    +		ActorRef taskManagerActorRef;
    +		if (taskManagerActorName != null && !taskManagerActorName.equals("")) {
    +			taskManagerActorRef = actorSystem.actorOf(tmProps, taskManagerActorName);
    +		} else {
    +			taskManagerActorRef = actorSystem.actorOf(tmProps);
    +		}
    +
    +		return taskManagerActorRef;
    +	}
    +
    +	// --------------------------------------------------------------------------
    +	//  Parsing and checking the TaskManager Configuration
    +	// --------------------------------------------------------------------------
    +
    +	/**
    +	 * Utility method to extract TaskManager config parameters from the configuration and to
    +	 * sanity check them.
    +	 *
    +	 * @param configuration                 The configuration.
    +	 * @param taskManagerHostname           The host name under which the TaskManager communicates.
    +	 * @param localTaskManagerCommunication True, to skip initializing the network stack.
    +	 *                                      Use only in cases where only one task manager runs.
    +	 * @return A tuple (TaskManagerConfiguration, network configuration,
    +	 * InstanceConnectionInfo, JobManager actor Akka URL).
    +	 */
    +	private static Tuple4<TaskManagerConfiguration, NetworkEnvironmentConfiguration, InstanceConnectionInfo, MemoryType>
    +		parseTaskManagerConfiguration(Configuration configuration, String taskManagerHostname, boolean localTaskManagerCommunication)
    +		throws Exception {
    +
    +		// ------- read values from the config and check them ---------
    +		//                      (a lot of them)
    +
    +		// ----> hosts / ports for communication and data exchange
    +
    +		int dataport = configuration.getInteger(ConfigConstants.TASK_MANAGER_DATA_PORT_KEY,
    +			ConfigConstants.DEFAULT_TASK_MANAGER_DATA_PORT);
    +		if (dataport == 0) {
    +			dataport = NetUtils.getAvailablePort();
    +		}
    +		checkConfigParameter(dataport > 0, dataport, ConfigConstants.TASK_MANAGER_DATA_PORT_KEY,
    +			"Leave config parameter empty or use 0 to let the system choose a port automatically.");
    +
    +		InetAddress taskManagerAddress = InetAddress.getByName(taskManagerHostname);
    +		InstanceConnectionInfo connectionInfo = new InstanceConnectionInfo(taskManagerAddress, dataport);
    +
    +		// ----> memory / network stack (shuffles/broadcasts), task slots, temp directories
    +
    +		// we need this because many configs have been written with a "-1" entry
    +		int slots = configuration.getInteger(ConfigConstants.TASK_MANAGER_NUM_TASK_SLOTS, 1);
    +		if (slots == -1) {
    +			slots = 1;
    +		}
    +		checkConfigParameter(slots >= 1, slots, ConfigConstants.TASK_MANAGER_NUM_TASK_SLOTS,
    +			"Number of task slots must be at least one.");
    +
    +		int numNetworkBuffers = configuration.getInteger(
    +			ConfigConstants.TASK_MANAGER_NETWORK_NUM_BUFFERS_KEY,
    +			ConfigConstants.DEFAULT_TASK_MANAGER_NETWORK_NUM_BUFFERS);
    +		checkConfigParameter(numNetworkBuffers > 0, numNetworkBuffers,
    +			ConfigConstants.TASK_MANAGER_NETWORK_NUM_BUFFERS_KEY, "");
    +
    +		int pageSize = configuration.getInteger(
    +			ConfigConstants.TASK_MANAGER_MEMORY_SEGMENT_SIZE_KEY,
    +			ConfigConstants.DEFAULT_TASK_MANAGER_MEMORY_SEGMENT_SIZE);
    +		// check page size of for minimum size
    +		checkConfigParameter(pageSize >= MemoryManager.MIN_PAGE_SIZE, pageSize,
    +			ConfigConstants.TASK_MANAGER_MEMORY_SEGMENT_SIZE_KEY,
    +			"Minimum memory segment size is " + MemoryManager.MIN_PAGE_SIZE);
    +		// check page size for power of two
    +		checkConfigParameter(MathUtils.isPowerOf2(pageSize), pageSize,
    +			ConfigConstants.TASK_MANAGER_MEMORY_SEGMENT_SIZE_KEY,
    +			"Memory segment size must be a power of 2.");
    +
    +		// check whether we use heap or off-heap memory
    +		MemoryType memType;
    --- End diff --
    
    Could be declared `final` to make assignment explicit.


---
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.
---

[GitHub] flink pull request #2400: [FLINK-4363] Implement TaskManager basic startup o...

Posted by mxm <gi...@git.apache.org>.
Github user mxm commented on a diff in the pull request:

    https://github.com/apache/flink/pull/2400#discussion_r75649283
  
    --- Diff: flink-runtime/src/main/java/org/apache/flink/runtime/rpc/taskexecutor/TaskExecutor.java ---
    @@ -35,27 +79,634 @@
      */
     public class TaskExecutor extends RpcEndpoint<TaskExecutorGateway> {
     
    -	/** The unique resource ID of this TaskExecutor */
    +	private static final Logger LOG = LoggerFactory.getLogger(TaskExecutor.class);
    +
    +	/** Return code for critical errors during the runtime */
    +	private static final int RUNTIME_FAILURE_RETURN_CODE = 2;
    +
    +	/** The name of the TaskManager actor */
    +	private static final String TASK_MANAGER_NAME = "taskmanager";
    +
    +	/** The unique resource ID of this TaskManager */
     	private final ResourceID resourceID;
     
     	/** The access to the leader election and metadata storage services */
     	private final HighAvailabilityServices haServices;
     
    -	// --------- resource manager --------
    +	/** The task manager configuration */
    +	private final TaskManagerConfiguration taskManagerConfig;
    +
    +	/** The connection information of the task manager */
    +	private final InstanceConnectionInfo connectionInfo;
    +
    +	/** The I/O manager component in the task manager */
    +	private final IOManager ioManager;
    +
    +	/** The memory manager component in the task manager */
    +	private final MemoryManager memoryManager;
    +
    +	/** The network component in the task manager */
    +	private final NetworkEnvironment networkEnvironment;
    +
    +	/** The number of slots in the task manager, should be 1 for YARN */
    +	private final int numberOfSlots;
     
    +	// --------- resource manager --------
     	private TaskExecutorToResourceManagerConnection resourceManagerConnection;
     
     	// ------------------------------------------------------------------------
     
     	public TaskExecutor(
    +			TaskManagerConfiguration taskManagerConfig,
    +			ResourceID resourceID,
    +			InstanceConnectionInfo connectionInfo,
    +			MemoryManager memoryManager,
    +			IOManager ioManager,
    +			NetworkEnvironment networkEnvironment,
    +			int numberOfSlots,
     			RpcService rpcService,
    -			HighAvailabilityServices haServices,
    -			ResourceID resourceID) {
    +			HighAvailabilityServices haServices) {
     
     		super(rpcService);
     
    -		this.haServices = checkNotNull(haServices);
    +		this.taskManagerConfig = checkNotNull(taskManagerConfig);
     		this.resourceID = checkNotNull(resourceID);
    +		this.connectionInfo = checkNotNull(connectionInfo);
    +		this.memoryManager = checkNotNull(memoryManager);
    +		this.ioManager = checkNotNull(ioManager);
    +		this.networkEnvironment = checkNotNull(networkEnvironment);
    +		this.numberOfSlots = checkNotNull(numberOfSlots);
    +		this.haServices = checkNotNull(haServices);
    +	}
    +
    +	/**
    +	 * Starts and runs the TaskManager.
    +	 * <p/>
    +	 * This method first tries to select the network interface to use for the TaskManager
    +	 * communication. The network interface is used both for the actor communication
    +	 * (coordination) as well as for the data exchange between task managers. Unless
    +	 * the hostname/interface is explicitly configured in the configuration, this
    +	 * method will try out various interfaces and methods to connect to the JobManager
    +	 * and select the one where the connection attempt is successful.
    +	 * <p/>
    +	 * After selecting the network interface, this method brings up an actor system
    +	 * for the TaskManager and its actors, starts the TaskManager's services
    +	 * (library cache, shuffle network stack, ...), and starts the TaskManager itself.
    +	 *
    +	 * @param configuration    The configuration for the TaskManager.
    +	 * @param taskManagerClass The actor class to instantiate.
    +	 *                         Allows to use TaskManager subclasses for example for YARN.
    +	 */
    +	public static void selectNetworkInterfaceAndRunTaskManager(
    +		Configuration configuration,
    +		ResourceID resourceID,
    +		Class<? extends TaskManager> taskManagerClass) throws Exception {
    +
    +		Tuple2<String, Integer> tuple2 = selectNetworkInterfaceAndPort(configuration);
    +
    +		runTaskManager(tuple2._1(), resourceID, tuple2._2(), configuration, taskManagerClass);
    +	}
    +
    +	private static Tuple2<String, Integer> selectNetworkInterfaceAndPort(Configuration configuration)
    +		throws Exception {
    +		String taskManagerHostname = configuration.getString(ConfigConstants.TASK_MANAGER_HOSTNAME_KEY, null);
    +		if (taskManagerHostname != null) {
    +			LOG.info("Using configured hostname/address for TaskManager: " + taskManagerHostname);
    +		} else {
    +			LeaderRetrievalService leaderRetrievalService = LeaderRetrievalUtils.createLeaderRetrievalService(configuration);
    +			FiniteDuration lookupTimeout = AkkaUtils.getLookupTimeout(configuration);
    +
    +			InetAddress taskManagerAddress = LeaderRetrievalUtils.findConnectingAddress(leaderRetrievalService, lookupTimeout);
    +			taskManagerHostname = taskManagerAddress.getHostName();
    +			LOG.info("TaskManager will use hostname/address '{}' ({}) for communication.",
    +				taskManagerHostname, taskManagerAddress.getHostAddress());
    +		}
    +
    +		// if no task manager port has been configured, use 0 (system will pick any free port)
    +		int actorSystemPort = configuration.getInteger(ConfigConstants.TASK_MANAGER_IPC_PORT_KEY, 0);
    +		if (actorSystemPort < 0 || actorSystemPort > 65535) {
    +			throw new IllegalConfigurationException("Invalid value for '" +
    +				ConfigConstants.TASK_MANAGER_IPC_PORT_KEY +
    +				"' (port for the TaskManager actor system) : " + actorSystemPort +
    +				" - Leave config parameter empty or use 0 to let the system choose a port automatically.");
    +		}
    +
    +		return new Tuple2<>(taskManagerHostname, actorSystemPort);
    +	}
    +
    +	/**
    +	 * Starts and runs the TaskManager. Brings up an actor system for the TaskManager and its
    +	 * actors, starts the TaskManager's services (library cache, shuffle network stack, ...),
    +	 * and starts the TaskManager itself.
    +	 * <p/>
    +	 * This method will also spawn a process reaper for the TaskManager (kill the process if
    +	 * the actor fails) and optionally start the JVM memory logging thread.
    +	 *
    +	 * @param taskManagerHostname The hostname/address of the interface where the actor system
    +	 *                            will communicate.
    +	 * @param resourceID          The id of the resource which the task manager will run on.
    +	 * @param actorSystemPort     The port at which the actor system will communicate.
    +	 * @param configuration       The configuration for the TaskManager.
    +	 * @param taskManagerClass    The actor class to instantiate. Allows the use of TaskManager
    +	 *                            subclasses for example for YARN.
    +	 */
    +	private static void runTaskManager(
    +		String taskManagerHostname,
    +		ResourceID resourceID,
    +		int actorSystemPort,
    +		Configuration configuration,
    +		Class<? extends TaskManager> taskManagerClass) throws Exception {
    +
    +		LOG.info("Starting TaskManager");
    +
    +		// Bring up the TaskManager actor system first, bind it to the given address.
    +
    +		LOG.info("Starting TaskManager actor system at " +
    +			NetUtils.hostAndPortToUrlString(taskManagerHostname, actorSystemPort));
    +
    +		ActorSystem taskManagerSystem;
    +		try {
    +			Tuple2<String, Object> address = new Tuple2<String, Object>(taskManagerHostname, actorSystemPort);
    +			Config akkaConfig = AkkaUtils.getAkkaConfig(configuration, new Some<>(address));
    +			LOG.debug("Using akka configuration\n " + akkaConfig);
    +			taskManagerSystem = AkkaUtils.createActorSystem(akkaConfig);
    +		} catch (Throwable t) {
    +			if (t instanceof org.jboss.netty.channel.ChannelException) {
    +				Throwable cause = t.getCause();
    +				if (cause != null && t.getCause() instanceof java.net.BindException) {
    +					String address = NetUtils.hostAndPortToUrlString(taskManagerHostname, actorSystemPort);
    +					throw new IOException("Unable to bind TaskManager actor system to address " +
    +						address + " - " + cause.getMessage(), t);
    +				}
    +			}
    +			throw new Exception("Could not create TaskManager actor system", t);
    +		}
    +
    +		// start all the TaskManager services (network stack,  library cache, ...)
    +		// and the TaskManager actor
    +		try {
    +			LOG.info("Starting TaskManager actor");
    +			ActorRef taskManagerActor = startTaskManagerComponentsAndActor(
    +				configuration,
    +				resourceID,
    +				taskManagerSystem,
    +				taskManagerHostname,
    +				TASK_MANAGER_NAME,
    +				null,
    +				false,
    +				taskManagerClass);
    +
    +			// start a process reaper that watches the JobManager. If the TaskManager actor dies,
    +			// the process reaper will kill the JVM process (to ensure easy failure detection)
    +			LOG.debug("Starting TaskManager process reaper");
    +
    +			taskManagerSystem.actorOf(
    +				Props.create(ProcessReaper.class, taskManagerActor, LOG, RUNTIME_FAILURE_RETURN_CODE),
    +				"TaskManager_Process_Reaper");
    +
    +			// if desired, start the logging daemon that periodically logs the memory usage information
    +			if (LOG.isInfoEnabled() && configuration.getBoolean(
    +				ConfigConstants.TASK_MANAGER_DEBUG_MEMORY_USAGE_START_LOG_THREAD,
    +				ConfigConstants.DEFAULT_TASK_MANAGER_DEBUG_MEMORY_USAGE_START_LOG_THREAD)) {
    +				LOG.info("Starting periodic memory usage logger");
    +
    +				long interval = configuration.getLong(
    +					ConfigConstants.TASK_MANAGER_DEBUG_MEMORY_USAGE_LOG_INTERVAL_MS,
    +					ConfigConstants.DEFAULT_TASK_MANAGER_DEBUG_MEMORY_USAGE_LOG_INTERVAL_MS);
    +
    +				MemoryLogger logger = new MemoryLogger(LOG, interval, taskManagerSystem);
    +				logger.start();
    +			}
    +
    +			// block until everything is done
    +			taskManagerSystem.awaitTermination();
    +		} catch (Throwable t) {
    +			LOG.error("Error while starting up taskManager", t);
    +			try {
    +				taskManagerSystem.shutdown();
    +			} catch (Throwable tt) {
    +				LOG.warn("Could not cleanly shut down actor system", tt);
    +			}
    +			throw t;
    +		}
    +	}
    +
    +	// --------------------------------------------------------------------------
    +	//  Starting and running the TaskManager
    +	// --------------------------------------------------------------------------
    +
    +	/**
    +	 * @param configuration                 The configuration for the TaskManager.
    +	 * @param resourceID                    The id of the resource which the task manager will run on.
    +	 * @param actorSystem                  The actor system that should run the TaskManager actor.
    +	 * @param taskManagerHostname       The hostname/address that describes the TaskManager's data location.
    +	 * @param taskManagerActorName      Optionally the name of the TaskManager actor. If none is given,
    +	 *                                      the actor will use a random name.
    +	 * @param leaderRetrievalService        Optionally, a leader retrieval service can be provided. If none is given,
    +	 *                                      then a LeaderRetrievalService is constructed from the configuration.
    +	 * @param localTaskManagerCommunication If true, the TaskManager will not initiate the TCP network stack.
    +	 * @param taskManagerClass      The class of the TaskManager actor. May be used to give
    +	 *                                      subclasses that understand additional actor messages.
    +	 * @return An ActorRef to the TaskManager actor.
    +	 * @throws org.apache.flink.configuration.IllegalConfigurationException     Thrown, if the given config contains illegal values.
    +	 * @throws java.io.IOException      Thrown, if any of the I/O components (such as buffer pools,
    +	 *                                       I/O manager, ...) cannot be properly started.
    +	 * @throws java.lang.Exception      Thrown is some other error occurs while parsing the configuration
    +	 *                                      or starting the TaskManager components.
    +	 */
    +	public static ActorRef startTaskManagerComponentsAndActor(
    +		Configuration configuration,
    +		ResourceID resourceID,
    +		ActorSystem actorSystem,
    +		String taskManagerHostname,
    +		String taskManagerActorName,
    +		LeaderRetrievalService leaderRetrievalService,
    +		boolean localTaskManagerCommunication,
    +		Class<? extends TaskManager> taskManagerClass) throws Exception {
    +
    +		Tuple4<TaskManagerConfiguration, NetworkEnvironmentConfiguration, InstanceConnectionInfo, MemoryType> tuple4
    +			= parseTaskManagerConfiguration(configuration, taskManagerHostname, localTaskManagerCommunication);
    +
    +		TaskManagerConfiguration taskManagerConfig = tuple4._1();
    +		NetworkEnvironmentConfiguration netConfig = tuple4._2();
    +		InstanceConnectionInfo connectionInfo = tuple4._3();
    +		MemoryType memType = tuple4._4();
    +
    +		// pre-start checks
    +		checkTempDirs(taskManagerConfig.tmpDirPaths());
    +
    +		ExecutionContext executionContext = ExecutionContexts$.MODULE$.fromExecutor(new ForkJoinPool());
    +
    +		// we start the network first, to make sure it can allocate its buffers first
    +		NetworkEnvironment network = new NetworkEnvironment(executionContext, taskManagerConfig.timeout(), netConfig);
    +
    +		// computing the amount of memory to use depends on how much memory is available
    +		// it strictly needs to happen AFTER the network stack has been initialized
    +
    +		// check if a value has been configured
    +		long configuredMemory = configuration.getLong(ConfigConstants.TASK_MANAGER_MEMORY_SIZE_KEY, -1L);
    +		checkConfigParameter(configuredMemory == -1 || configuredMemory > 0, configuredMemory,
    +			ConfigConstants.TASK_MANAGER_MEMORY_SIZE_KEY,
    +			"MemoryManager needs at least one MB of memory. " +
    +				"If you leave this config parameter empty, the system automatically " +
    +				"pick a fraction of the available memory.");
    +
    +		long memorySize;
    +		boolean preAllocateMemory = configuration.getBoolean(
    +			ConfigConstants.TASK_MANAGER_MEMORY_PRE_ALLOCATE_KEY,
    +			ConfigConstants.DEFAULT_TASK_MANAGER_MEMORY_PRE_ALLOCATE);
    +		if (configuredMemory > 0) {
    +			if (preAllocateMemory) {
    +				LOG.info("Using {} MB for managed memory." , configuredMemory);
    +			} else {
    +				LOG.info("Limiting managed memory to {} MB, memory will be allocated lazily." , configuredMemory);
    +			}
    +			memorySize = configuredMemory << 20; // megabytes to bytes
    +		} else {
    +			float fraction = configuration.getFloat(
    +				ConfigConstants.TASK_MANAGER_MEMORY_FRACTION_KEY,
    +				ConfigConstants.DEFAULT_MEMORY_MANAGER_MEMORY_FRACTION);
    +			checkConfigParameter(fraction > 0.0f && fraction < 1.0f, fraction,
    +				ConfigConstants.TASK_MANAGER_MEMORY_FRACTION_KEY,
    +				"MemoryManager fraction of the free memory must be between 0.0 and 1.0");
    +
    +			if (memType == MemoryType.HEAP) {
    +				long relativeMemSize = (long) (EnvironmentInformation.getSizeOfFreeHeapMemoryWithDefrag() * fraction);
    +				if (preAllocateMemory) {
    +					LOG.info("Using {} of the currently free heap space for managed heap memory ({} MB)." ,
    +						fraction , relativeMemSize >> 20);
    +				} else {
    +					LOG.info("Limiting managed memory to {} of the currently free heap space ({} MB), " +
    +						"memory will be allocated lazily." , fraction , relativeMemSize >> 20);
    +				}
    +				memorySize = relativeMemSize;
    +			} else if (memType == MemoryType.OFF_HEAP) {
    +				// The maximum heap memory has been adjusted according to the fraction
    +				long maxMemory = EnvironmentInformation.getMaxJvmHeapMemory();
    +				long directMemorySize = (long) (maxMemory / (1.0 - fraction) * fraction);
    +				if (preAllocateMemory) {
    +					LOG.info("Using {} of the maximum memory size for managed off-heap memory ({} MB)." ,
    +						fraction, directMemorySize >> 20);
    +				} else {
    +					LOG.info("Limiting managed memory to {} of the maximum memory size ({} MB)," +
    +						" memory will be allocated lazily.", fraction, directMemorySize >> 20);
    +				}
    +				memorySize = directMemorySize;
    +			} else {
    +				throw new RuntimeException("No supported memory type detected.");
    +			}
    +		}
    +
    +		// now start the memory manager
    +		MemoryManager memoryManager;
    +		try {
    +			memoryManager = new MemoryManager(
    +				memorySize,
    +				taskManagerConfig.numberOfSlots(),
    +				netConfig.networkBufferSize(),
    +				memType,
    +				preAllocateMemory);
    +		} catch (OutOfMemoryError e) {
    +			if (memType == MemoryType.HEAP) {
    +				throw new Exception("OutOfMemory error (" + e.getMessage() +
    +					") while allocating the TaskManager heap memory (" + memorySize + " bytes).", e);
    +			} else if (memType == MemoryType.OFF_HEAP) {
    +				throw new Exception("OutOfMemory error (" + e.getMessage() +
    +					") while allocating the TaskManager off-heap memory (" + memorySize +
    +					" bytes).Try increasing the maximum direct memory (-XX:MaxDirectMemorySize)", e);
    +			} else {
    +				throw e;
    +			}
    +		}
    +
    +		// start the I/O manager, it will create some temp directories.
    +		IOManager ioManager = new IOManagerAsync(taskManagerConfig.tmpDirPaths());
    +
    +		if (leaderRetrievalService == null){
    +			leaderRetrievalService = LeaderRetrievalUtils.createLeaderRetrievalService(configuration);
    +		}
    +
    +		// create the actor properties (which define the actor constructor parameters)
    +		Props tmProps = Props.create(
    +			taskManagerClass,
    +			taskManagerConfig,
    +			resourceID,
    +			connectionInfo,
    +			memoryManager,
    +			ioManager,
    +			network,
    +			taskManagerConfig.numberOfSlots(),
    +			leaderRetrievalService);
    +
    +		ActorRef taskManagerActorRef;
    --- End diff --
    
    `final` 


---
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.
---

[GitHub] flink pull request #2400: [FLINK-4363] Implement TaskManager basic startup o...

Posted by mxm <gi...@git.apache.org>.
Github user mxm commented on a diff in the pull request:

    https://github.com/apache/flink/pull/2400#discussion_r76212796
  
    --- Diff: flink-runtime/src/main/java/org/apache/flink/runtime/rpc/taskexecutor/TaskExecutor.java ---
    @@ -36,27 +82,617 @@
      */
     public class TaskExecutor extends RpcEndpoint<TaskExecutorGateway> {
     
    -	/** The unique resource ID of this TaskExecutor */
    +	private static final Logger LOG = LoggerFactory.getLogger(TaskExecutor.class);
    +
    +	/** The unique resource ID of this TaskManager */
     	private final ResourceID resourceID;
     
     	/** The access to the leader election and metadata storage services */
     	private final HighAvailabilityServices haServices;
     
    -	// --------- resource manager --------
    +	/** The task manager configuration */
    +	private final TaskExecutorConfiguration taskExecutorConfig;
    +
    +	/** The connection information of the task manager */
    +	private final InstanceConnectionInfo connectionInfo;
    +
    +	/** The I/O manager component in the task manager */
    +	private final IOManager ioManager;
    +
    +	/** The memory manager component in the task manager */
    +	private final MemoryManager memoryManager;
    +
    +	/** The network component in the task manager */
    +	private final NetworkEnvironment networkEnvironment;
    +
    +	/** The number of slots in the task manager, should be 1 for YARN */
    +	private final int numberOfSlots;
     
    +	// --------- resource manager --------
     	private TaskExecutorToResourceManagerConnection resourceManagerConnection;
     
     	// ------------------------------------------------------------------------
     
     	public TaskExecutor(
    +			TaskExecutorConfiguration taskExecutorConfig,
    +			ResourceID resourceID,
    +			InstanceConnectionInfo connectionInfo,
    +			MemoryManager memoryManager,
    +			IOManager ioManager,
    +			NetworkEnvironment networkEnvironment,
    +			int numberOfSlots,
     			RpcService rpcService,
    -			HighAvailabilityServices haServices,
    -			ResourceID resourceID) {
    +			HighAvailabilityServices haServices) {
     
     		super(rpcService);
     
    -		this.haServices = checkNotNull(haServices);
    +		this.taskExecutorConfig = checkNotNull(taskExecutorConfig);
     		this.resourceID = checkNotNull(resourceID);
    +		this.connectionInfo = checkNotNull(connectionInfo);
    +		this.memoryManager = checkNotNull(memoryManager);
    +		this.ioManager = checkNotNull(ioManager);
    +		this.networkEnvironment = checkNotNull(networkEnvironment);
    +		this.numberOfSlots = checkNotNull(numberOfSlots);
    +		this.haServices = checkNotNull(haServices);
    +	}
    +
    +	/**
    +	 * Starts and runs the TaskManager.
    +	 * <p/>
    +	 * This method first tries to select the network interface to use for the TaskManager
    +	 * communication. The network interface is used both for the actor communication
    +	 * (coordination) as well as for the data exchange between task managers. Unless
    +	 * the hostname/interface is explicitly configured in the configuration, this
    +	 * method will try out various interfaces and methods to connect to the JobManager
    +	 * and select the one where the connection attempt is successful.
    +	 * <p/>
    +	 * After selecting the network interface, this method brings up an actor system
    +	 * for the TaskManager and its actors, starts the TaskManager's services
    +	 * (library cache, shuffle network stack, ...), and starts the TaskManager itself.
    +	 *
    +	 * @param configuration    The configuration for the TaskManager.
    +	 * @param resourceID       The id of the resource which the task manager will run on.
    +	 */
    +	public static void selectNetworkInterfaceAndRunTaskManager(
    +		Configuration configuration,
    +		ResourceID resourceID) throws Exception {
    +
    +		InetSocketAddress taskManagerAddress = selectNetworkInterfaceAndPort(configuration);
    +
    +		runTaskManager(taskManagerAddress.getHostName(), resourceID, taskManagerAddress.getPort(), configuration);
    +	}
    +
    +	private static InetSocketAddress selectNetworkInterfaceAndPort(Configuration configuration)
    +		throws Exception {
    +		String taskManagerHostname = configuration.getString(ConfigConstants.TASK_MANAGER_HOSTNAME_KEY, null);
    +		if (taskManagerHostname != null) {
    +			LOG.info("Using configured hostname/address for TaskManager: " + taskManagerHostname);
    +		} else {
    +			LeaderRetrievalService leaderRetrievalService = LeaderRetrievalUtils.createLeaderRetrievalService(configuration);
    +			FiniteDuration lookupTimeout = AkkaUtils.getLookupTimeout(configuration);
    +
    +			InetAddress taskManagerAddress = LeaderRetrievalUtils.findConnectingAddress(leaderRetrievalService, lookupTimeout);
    +			taskManagerHostname = taskManagerAddress.getHostName();
    +			LOG.info("TaskManager will use hostname/address '{}' ({}) for communication.",
    +				taskManagerHostname, taskManagerAddress.getHostAddress());
    +		}
    +
    +		// if no task manager port has been configured, use 0 (system will pick any free port)
    +		int actorSystemPort = configuration.getInteger(ConfigConstants.TASK_MANAGER_IPC_PORT_KEY, 0);
    +		if (actorSystemPort < 0 || actorSystemPort > 65535) {
    +			throw new IllegalConfigurationException("Invalid value for '" +
    +				ConfigConstants.TASK_MANAGER_IPC_PORT_KEY +
    +				"' (port for the TaskManager actor system) : " + actorSystemPort +
    +				" - Leave config parameter empty or use 0 to let the system choose a port automatically.");
    +		}
    +
    +		return new InetSocketAddress(taskManagerHostname, actorSystemPort);
    +	}
    +
    +	/**
    +	 * Starts and runs the TaskManager. Brings up an actor system for the TaskManager and its
    +	 * actors, starts the TaskManager's services (library cache, shuffle network stack, ...),
    +	 * and starts the TaskManager itself.
    +	 * <p/>
    +	 * This method will also spawn a process reaper for the TaskManager (kill the process if
    +	 * the actor fails) and optionally start the JVM memory logging thread.
    +	 *
    +	 * @param taskManagerHostname The hostname/address of the interface where the actor system
    +	 *                            will communicate.
    +	 * @param resourceID          The id of the resource which the task manager will run on.
    +	 * @param actorSystemPort   The port at which the actor system will communicate.
    +	 * @param configuration       The configuration for the TaskManager.
    +	 */
    +	private static void runTaskManager(
    +		String taskManagerHostname,
    +		ResourceID resourceID,
    +		int actorSystemPort,
    +		final Configuration configuration) throws Exception {
    +
    +		LOG.info("Starting TaskManager");
    +
    +		// Bring up the TaskManager actor system first, bind it to the given address.
    +
    +		LOG.info("Starting TaskManager actor system at " +
    +			NetUtils.hostAndPortToUrlString(taskManagerHostname, actorSystemPort));
    +
    +		final ActorSystem taskManagerSystem;
    +		try {
    +			Tuple2<String, Object> address = new Tuple2<String, Object>(taskManagerHostname, actorSystemPort);
    +			Config akkaConfig = AkkaUtils.getAkkaConfig(configuration, new Some<>(address));
    +			LOG.debug("Using akka configuration\n " + akkaConfig);
    +			taskManagerSystem = AkkaUtils.createActorSystem(akkaConfig);
    +		} catch (Throwable t) {
    +			if (t instanceof org.jboss.netty.channel.ChannelException) {
    +				Throwable cause = t.getCause();
    +				if (cause != null && t.getCause() instanceof java.net.BindException) {
    +					String address = NetUtils.hostAndPortToUrlString(taskManagerHostname, actorSystemPort);
    +					throw new IOException("Unable to bind TaskManager actor system to address " +
    +						address + " - " + cause.getMessage(), t);
    +				}
    +			}
    +			throw new Exception("Could not create TaskManager actor system", t);
    +		}
    +
    +		// start akka rpc service based on actor system
    +		Timeout timeout = new Timeout(AkkaUtils.getTimeout(configuration).toMillis(), TimeUnit.MILLISECONDS);
    +		AkkaRpcService akkaRpcService = new AkkaRpcService(taskManagerSystem, timeout);
    +
    +		// start high availability service to implement getResourceManagerLeaderRetriever method only
    +		HighAvailabilityServices haServices = new HighAvailabilityServices() {
    +			@Override
    +			public LeaderRetrievalService getResourceManagerLeaderRetriever() throws Exception {
    +				return LeaderRetrievalUtils.createLeaderRetrievalService(configuration);
    +			}
    +
    +			@Override
    +			public LeaderElectionService getJobMasterLeaderElectionService(JobID jobID) throws Exception {
    +				return null;
    +			}
    +		};
    +
    +		// start all the TaskManager services (network stack,  library cache, ...)
    +		// and the TaskManager actor
    +		try {
    +			LOG.info("Starting TaskManager actor");
    +			TaskExecutor taskExecutor = startTaskManagerComponentsAndActor(
    +				configuration,
    +				resourceID,
    +				akkaRpcService,
    +				taskManagerHostname,
    +				haServices,
    +				false);
    +
    +			taskExecutor.start();
    +
    +			// if desired, start the logging daemon that periodically logs the memory usage information
    +			if (LOG.isInfoEnabled() && configuration.getBoolean(
    +				ConfigConstants.TASK_MANAGER_DEBUG_MEMORY_USAGE_START_LOG_THREAD,
    +				ConfigConstants.DEFAULT_TASK_MANAGER_DEBUG_MEMORY_USAGE_START_LOG_THREAD)) {
    +				LOG.info("Starting periodic memory usage logger");
    +
    +				long interval = configuration.getLong(
    +					ConfigConstants.TASK_MANAGER_DEBUG_MEMORY_USAGE_LOG_INTERVAL_MS,
    +					ConfigConstants.DEFAULT_TASK_MANAGER_DEBUG_MEMORY_USAGE_LOG_INTERVAL_MS);
    +
    +				MemoryLogger logger = new MemoryLogger(LOG, interval, taskManagerSystem);
    +				logger.start();
    +			}
    +
    +			// block until everything is done
    +			taskManagerSystem.awaitTermination();
    +		} catch (Throwable t) {
    +			LOG.error("Error while starting up taskManager", t);
    +			try {
    +				taskManagerSystem.shutdown();
    +			} catch (Throwable tt) {
    +				LOG.warn("Could not cleanly shut down actor system", tt);
    +			}
    +			throw t;
    +		}
    +	}
    +
    +	// --------------------------------------------------------------------------
    +	//  Starting and running the TaskManager
    +	// --------------------------------------------------------------------------
    +
    +	/**
    +	 * @param configuration                 The configuration for the TaskManager.
    +	 * @param resourceID                    The id of the resource which the task manager will run on.
    +	 * @param rpcService                  The rpc service which is used to start and connect to the TaskManager RpcEndpoint .
    +	 * @param taskManagerHostname       The hostname/address that describes the TaskManager's data location.
    +	 * @param haServices        Optionally, a high availability service can be provided. If none is given,
    +	 *                                      then a HighAvailabilityServices is constructed from the configuration.
    +	 * @param localTaskManagerCommunication If true, the TaskManager will not initiate the TCP network stack.
    +	 * @return An ActorRef to the TaskManager actor.
    +	 * @throws org.apache.flink.configuration.IllegalConfigurationException     Thrown, if the given config contains illegal values.
    +	 * @throws java.io.IOException      Thrown, if any of the I/O components (such as buffer pools,
    +	 *                                       I/O manager, ...) cannot be properly started.
    +	 * @throws java.lang.Exception      Thrown is some other error occurs while parsing the configuration
    +	 *                                      or starting the TaskManager components.
    +	 */
    +	public static TaskExecutor startTaskManagerComponentsAndActor(
    +		Configuration configuration,
    +		ResourceID resourceID,
    +		RpcService rpcService,
    +		String taskManagerHostname,
    +		HighAvailabilityServices haServices,
    +		boolean localTaskManagerCommunication) throws Exception {
    +
    +		Tuple4<TaskExecutorConfiguration, NetworkEnvironmentConfiguration, InstanceConnectionInfo, MemoryType> tuple4
    +			= parseTaskManagerConfiguration(configuration, taskManagerHostname, localTaskManagerCommunication);
    +
    +		TaskExecutorConfiguration taskExecutorConfig = tuple4.f0;
    +		NetworkEnvironmentConfiguration netConfig = tuple4.f1;
    +		InstanceConnectionInfo connectionInfo = tuple4.f2;
    +		MemoryType memType = tuple4.f3;
    +
    +		// pre-start checks
    +		checkTempDirs(taskExecutorConfig.getTmpDirPaths());
    +
    +		ExecutionContext executionContext = ExecutionContexts$.MODULE$.fromExecutor(new ForkJoinPool());
    +
    +		// we start the network first, to make sure it can allocate its buffers first
    +		NetworkEnvironment network = new NetworkEnvironment(executionContext, taskExecutorConfig.getTimeout(), netConfig);
    +
    +		// computing the amount of memory to use depends on how much memory is available
    +		// it strictly needs to happen AFTER the network stack has been initialized
    +
    +		// check if a value has been configured
    +		long configuredMemory = configuration.getLong(ConfigConstants.TASK_MANAGER_MEMORY_SIZE_KEY, -1L);
    +		checkConfigParameter(configuredMemory == -1 || configuredMemory > 0, configuredMemory,
    +			ConfigConstants.TASK_MANAGER_MEMORY_SIZE_KEY,
    +			"MemoryManager needs at least one MB of memory. " +
    +				"If you leave this config parameter empty, the system automatically " +
    +				"pick a fraction of the available memory.");
    +
    +		long memorySize;
    +		boolean preAllocateMemory = configuration.getBoolean(
    +			ConfigConstants.TASK_MANAGER_MEMORY_PRE_ALLOCATE_KEY,
    +			ConfigConstants.DEFAULT_TASK_MANAGER_MEMORY_PRE_ALLOCATE);
    +		if (configuredMemory > 0) {
    +			if (preAllocateMemory) {
    +				LOG.info("Using {} MB for managed memory." , configuredMemory);
    +			} else {
    +				LOG.info("Limiting managed memory to {} MB, memory will be allocated lazily." , configuredMemory);
    +			}
    +			memorySize = configuredMemory << 20; // megabytes to bytes
    +		} else {
    +			float fraction = configuration.getFloat(
    +				ConfigConstants.TASK_MANAGER_MEMORY_FRACTION_KEY,
    +				ConfigConstants.DEFAULT_MEMORY_MANAGER_MEMORY_FRACTION);
    +			checkConfigParameter(fraction > 0.0f && fraction < 1.0f, fraction,
    +				ConfigConstants.TASK_MANAGER_MEMORY_FRACTION_KEY,
    +				"MemoryManager fraction of the free memory must be between 0.0 and 1.0");
    +
    +			if (memType == MemoryType.HEAP) {
    +				long relativeMemSize = (long) (EnvironmentInformation.getSizeOfFreeHeapMemoryWithDefrag() * fraction);
    +				if (preAllocateMemory) {
    +					LOG.info("Using {} of the currently free heap space for managed heap memory ({} MB)." ,
    +						fraction , relativeMemSize >> 20);
    +				} else {
    +					LOG.info("Limiting managed memory to {} of the currently free heap space ({} MB), " +
    +						"memory will be allocated lazily." , fraction , relativeMemSize >> 20);
    +				}
    +				memorySize = relativeMemSize;
    +			} else if (memType == MemoryType.OFF_HEAP) {
    +				// The maximum heap memory has been adjusted according to the fraction
    +				long maxMemory = EnvironmentInformation.getMaxJvmHeapMemory();
    +				long directMemorySize = (long) (maxMemory / (1.0 - fraction) * fraction);
    +				if (preAllocateMemory) {
    +					LOG.info("Using {} of the maximum memory size for managed off-heap memory ({} MB)." ,
    +						fraction, directMemorySize >> 20);
    +				} else {
    +					LOG.info("Limiting managed memory to {} of the maximum memory size ({} MB)," +
    +						" memory will be allocated lazily.", fraction, directMemorySize >> 20);
    +				}
    +				memorySize = directMemorySize;
    +			} else {
    +				throw new RuntimeException("No supported memory type detected.");
    +			}
    +		}
    +
    +		// now start the memory manager
    +		final MemoryManager memoryManager;
    +		try {
    +			memoryManager = new MemoryManager(
    +				memorySize,
    +				taskExecutorConfig.getNumberOfSlots(),
    +				netConfig.networkBufferSize(),
    +				memType,
    +				preAllocateMemory);
    +		} catch (OutOfMemoryError e) {
    +			if (memType == MemoryType.HEAP) {
    +				throw new Exception("OutOfMemory error (" + e.getMessage() +
    +					") while allocating the TaskManager heap memory (" + memorySize + " bytes).", e);
    +			} else if (memType == MemoryType.OFF_HEAP) {
    +				throw new Exception("OutOfMemory error (" + e.getMessage() +
    +					") while allocating the TaskManager off-heap memory (" + memorySize +
    +					" bytes).Try increasing the maximum direct memory (-XX:MaxDirectMemorySize)", e);
    +			} else {
    +				throw e;
    +			}
    +		}
    +
    +		// start the I/O manager, it will create some temp directories.
    +		IOManager ioManager = new IOManagerAsync(taskExecutorConfig.getTmpDirPaths());
    +
    +		TaskExecutor taskExecutor = new TaskExecutor(
    +			taskExecutorConfig,
    +			resourceID,
    +			connectionInfo,
    +			memoryManager,
    +			ioManager,
    +			network,
    +			taskExecutorConfig.getNumberOfSlots(),
    +			rpcService,
    +			haServices);
    +
    +		return taskExecutor;
    +	}
    +
    +	// --------------------------------------------------------------------------
    +	//  Parsing and checking the TaskManager Configuration
    +	// --------------------------------------------------------------------------
    +
    +	/**
    +	 * Utility method to extract TaskManager config parameters from the configuration and to
    +	 * sanity check them.
    +	 *
    +	 * @param configuration                 The configuration.
    +	 * @param taskManagerHostname           The host name under which the TaskManager communicates.
    +	 * @param localTaskManagerCommunication True, to skip initializing the network stack.
    +	 *                                      Use only in cases where only one task manager runs.
    +	 * @return A tuple (TaskManagerConfiguration, network configuration,
    +	 * InstanceConnectionInfo, JobManager actor Akka URL).
    +	 */
    +	private static Tuple4<TaskExecutorConfiguration, NetworkEnvironmentConfiguration, InstanceConnectionInfo, MemoryType>
    --- End diff --
    
    Yes, the `TaskExecutorFactory` should be an abstract class with the helper methods to bring up the TaskManager. The factory can be implemented by some classes to start a TaskManager in different modes (testing, standalone, yarn). 
    
    If you want we can also merge this first and work on this in a follow-up. It is not the most pressing issue.


---
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.
---

[GitHub] flink pull request #2400: [FLINK-4363] Implement TaskManager basic startup o...

Posted by mxm <gi...@git.apache.org>.
Github user mxm commented on a diff in the pull request:

    https://github.com/apache/flink/pull/2400#discussion_r75648030
  
    --- Diff: flink-runtime/src/main/java/org/apache/flink/runtime/rpc/taskexecutor/TaskExecutor.java ---
    @@ -35,27 +79,634 @@
      */
     public class TaskExecutor extends RpcEndpoint<TaskExecutorGateway> {
     
    -	/** The unique resource ID of this TaskExecutor */
    +	private static final Logger LOG = LoggerFactory.getLogger(TaskExecutor.class);
    +
    +	/** Return code for critical errors during the runtime */
    +	private static final int RUNTIME_FAILURE_RETURN_CODE = 2;
    +
    +	/** The name of the TaskManager actor */
    +	private static final String TASK_MANAGER_NAME = "taskmanager";
    +
    +	/** The unique resource ID of this TaskManager */
     	private final ResourceID resourceID;
     
     	/** The access to the leader election and metadata storage services */
     	private final HighAvailabilityServices haServices;
     
    -	// --------- resource manager --------
    +	/** The task manager configuration */
    +	private final TaskManagerConfiguration taskManagerConfig;
    +
    +	/** The connection information of the task manager */
    +	private final InstanceConnectionInfo connectionInfo;
    +
    +	/** The I/O manager component in the task manager */
    +	private final IOManager ioManager;
    +
    +	/** The memory manager component in the task manager */
    +	private final MemoryManager memoryManager;
    +
    +	/** The network component in the task manager */
    +	private final NetworkEnvironment networkEnvironment;
    +
    +	/** The number of slots in the task manager, should be 1 for YARN */
    +	private final int numberOfSlots;
     
    +	// --------- resource manager --------
     	private TaskExecutorToResourceManagerConnection resourceManagerConnection;
     
     	// ------------------------------------------------------------------------
     
     	public TaskExecutor(
    +			TaskManagerConfiguration taskManagerConfig,
    +			ResourceID resourceID,
    +			InstanceConnectionInfo connectionInfo,
    +			MemoryManager memoryManager,
    +			IOManager ioManager,
    +			NetworkEnvironment networkEnvironment,
    +			int numberOfSlots,
     			RpcService rpcService,
    -			HighAvailabilityServices haServices,
    -			ResourceID resourceID) {
    +			HighAvailabilityServices haServices) {
     
     		super(rpcService);
     
    -		this.haServices = checkNotNull(haServices);
    +		this.taskManagerConfig = checkNotNull(taskManagerConfig);
     		this.resourceID = checkNotNull(resourceID);
    +		this.connectionInfo = checkNotNull(connectionInfo);
    +		this.memoryManager = checkNotNull(memoryManager);
    +		this.ioManager = checkNotNull(ioManager);
    +		this.networkEnvironment = checkNotNull(networkEnvironment);
    +		this.numberOfSlots = checkNotNull(numberOfSlots);
    +		this.haServices = checkNotNull(haServices);
    +	}
    +
    +	/**
    +	 * Starts and runs the TaskManager.
    +	 * <p/>
    +	 * This method first tries to select the network interface to use for the TaskManager
    +	 * communication. The network interface is used both for the actor communication
    +	 * (coordination) as well as for the data exchange between task managers. Unless
    +	 * the hostname/interface is explicitly configured in the configuration, this
    +	 * method will try out various interfaces and methods to connect to the JobManager
    +	 * and select the one where the connection attempt is successful.
    +	 * <p/>
    +	 * After selecting the network interface, this method brings up an actor system
    +	 * for the TaskManager and its actors, starts the TaskManager's services
    +	 * (library cache, shuffle network stack, ...), and starts the TaskManager itself.
    +	 *
    +	 * @param configuration    The configuration for the TaskManager.
    +	 * @param taskManagerClass The actor class to instantiate.
    +	 *                         Allows to use TaskManager subclasses for example for YARN.
    +	 */
    +	public static void selectNetworkInterfaceAndRunTaskManager(
    +		Configuration configuration,
    +		ResourceID resourceID,
    +		Class<? extends TaskManager> taskManagerClass) throws Exception {
    +
    +		Tuple2<String, Integer> tuple2 = selectNetworkInterfaceAndPort(configuration);
    +
    +		runTaskManager(tuple2._1(), resourceID, tuple2._2(), configuration, taskManagerClass);
    +	}
    +
    +	private static Tuple2<String, Integer> selectNetworkInterfaceAndPort(Configuration configuration)
    +		throws Exception {
    +		String taskManagerHostname = configuration.getString(ConfigConstants.TASK_MANAGER_HOSTNAME_KEY, null);
    +		if (taskManagerHostname != null) {
    +			LOG.info("Using configured hostname/address for TaskManager: " + taskManagerHostname);
    +		} else {
    +			LeaderRetrievalService leaderRetrievalService = LeaderRetrievalUtils.createLeaderRetrievalService(configuration);
    +			FiniteDuration lookupTimeout = AkkaUtils.getLookupTimeout(configuration);
    +
    +			InetAddress taskManagerAddress = LeaderRetrievalUtils.findConnectingAddress(leaderRetrievalService, lookupTimeout);
    +			taskManagerHostname = taskManagerAddress.getHostName();
    +			LOG.info("TaskManager will use hostname/address '{}' ({}) for communication.",
    +				taskManagerHostname, taskManagerAddress.getHostAddress());
    +		}
    +
    +		// if no task manager port has been configured, use 0 (system will pick any free port)
    +		int actorSystemPort = configuration.getInteger(ConfigConstants.TASK_MANAGER_IPC_PORT_KEY, 0);
    +		if (actorSystemPort < 0 || actorSystemPort > 65535) {
    +			throw new IllegalConfigurationException("Invalid value for '" +
    +				ConfigConstants.TASK_MANAGER_IPC_PORT_KEY +
    +				"' (port for the TaskManager actor system) : " + actorSystemPort +
    +				" - Leave config parameter empty or use 0 to let the system choose a port automatically.");
    +		}
    +
    +		return new Tuple2<>(taskManagerHostname, actorSystemPort);
    +	}
    +
    +	/**
    +	 * Starts and runs the TaskManager. Brings up an actor system for the TaskManager and its
    +	 * actors, starts the TaskManager's services (library cache, shuffle network stack, ...),
    +	 * and starts the TaskManager itself.
    +	 * <p/>
    +	 * This method will also spawn a process reaper for the TaskManager (kill the process if
    +	 * the actor fails) and optionally start the JVM memory logging thread.
    +	 *
    +	 * @param taskManagerHostname The hostname/address of the interface where the actor system
    +	 *                            will communicate.
    +	 * @param resourceID          The id of the resource which the task manager will run on.
    +	 * @param actorSystemPort     The port at which the actor system will communicate.
    +	 * @param configuration       The configuration for the TaskManager.
    +	 * @param taskManagerClass    The actor class to instantiate. Allows the use of TaskManager
    +	 *                            subclasses for example for YARN.
    +	 */
    +	private static void runTaskManager(
    +		String taskManagerHostname,
    +		ResourceID resourceID,
    +		int actorSystemPort,
    +		Configuration configuration,
    +		Class<? extends TaskManager> taskManagerClass) throws Exception {
    +
    +		LOG.info("Starting TaskManager");
    +
    +		// Bring up the TaskManager actor system first, bind it to the given address.
    +
    +		LOG.info("Starting TaskManager actor system at " +
    +			NetUtils.hostAndPortToUrlString(taskManagerHostname, actorSystemPort));
    +
    +		ActorSystem taskManagerSystem;
    +		try {
    +			Tuple2<String, Object> address = new Tuple2<String, Object>(taskManagerHostname, actorSystemPort);
    +			Config akkaConfig = AkkaUtils.getAkkaConfig(configuration, new Some<>(address));
    +			LOG.debug("Using akka configuration\n " + akkaConfig);
    +			taskManagerSystem = AkkaUtils.createActorSystem(akkaConfig);
    +		} catch (Throwable t) {
    +			if (t instanceof org.jboss.netty.channel.ChannelException) {
    +				Throwable cause = t.getCause();
    +				if (cause != null && t.getCause() instanceof java.net.BindException) {
    +					String address = NetUtils.hostAndPortToUrlString(taskManagerHostname, actorSystemPort);
    +					throw new IOException("Unable to bind TaskManager actor system to address " +
    +						address + " - " + cause.getMessage(), t);
    +				}
    +			}
    +			throw new Exception("Could not create TaskManager actor system", t);
    +		}
    +
    +		// start all the TaskManager services (network stack,  library cache, ...)
    +		// and the TaskManager actor
    +		try {
    +			LOG.info("Starting TaskManager actor");
    +			ActorRef taskManagerActor = startTaskManagerComponentsAndActor(
    +				configuration,
    +				resourceID,
    +				taskManagerSystem,
    +				taskManagerHostname,
    +				TASK_MANAGER_NAME,
    +				null,
    +				false,
    +				taskManagerClass);
    +
    +			// start a process reaper that watches the JobManager. If the TaskManager actor dies,
    +			// the process reaper will kill the JVM process (to ensure easy failure detection)
    +			LOG.debug("Starting TaskManager process reaper");
    +
    +			taskManagerSystem.actorOf(
    +				Props.create(ProcessReaper.class, taskManagerActor, LOG, RUNTIME_FAILURE_RETURN_CODE),
    +				"TaskManager_Process_Reaper");
    +
    +			// if desired, start the logging daemon that periodically logs the memory usage information
    +			if (LOG.isInfoEnabled() && configuration.getBoolean(
    +				ConfigConstants.TASK_MANAGER_DEBUG_MEMORY_USAGE_START_LOG_THREAD,
    +				ConfigConstants.DEFAULT_TASK_MANAGER_DEBUG_MEMORY_USAGE_START_LOG_THREAD)) {
    +				LOG.info("Starting periodic memory usage logger");
    +
    +				long interval = configuration.getLong(
    +					ConfigConstants.TASK_MANAGER_DEBUG_MEMORY_USAGE_LOG_INTERVAL_MS,
    +					ConfigConstants.DEFAULT_TASK_MANAGER_DEBUG_MEMORY_USAGE_LOG_INTERVAL_MS);
    +
    +				MemoryLogger logger = new MemoryLogger(LOG, interval, taskManagerSystem);
    +				logger.start();
    +			}
    +
    +			// block until everything is done
    +			taskManagerSystem.awaitTermination();
    +		} catch (Throwable t) {
    +			LOG.error("Error while starting up taskManager", t);
    +			try {
    +				taskManagerSystem.shutdown();
    +			} catch (Throwable tt) {
    +				LOG.warn("Could not cleanly shut down actor system", tt);
    +			}
    +			throw t;
    +		}
    +	}
    +
    +	// --------------------------------------------------------------------------
    +	//  Starting and running the TaskManager
    +	// --------------------------------------------------------------------------
    +
    +	/**
    +	 * @param configuration                 The configuration for the TaskManager.
    +	 * @param resourceID                    The id of the resource which the task manager will run on.
    +	 * @param actorSystem                  The actor system that should run the TaskManager actor.
    +	 * @param taskManagerHostname       The hostname/address that describes the TaskManager's data location.
    +	 * @param taskManagerActorName      Optionally the name of the TaskManager actor. If none is given,
    +	 *                                      the actor will use a random name.
    +	 * @param leaderRetrievalService        Optionally, a leader retrieval service can be provided. If none is given,
    +	 *                                      then a LeaderRetrievalService is constructed from the configuration.
    +	 * @param localTaskManagerCommunication If true, the TaskManager will not initiate the TCP network stack.
    +	 * @param taskManagerClass      The class of the TaskManager actor. May be used to give
    +	 *                                      subclasses that understand additional actor messages.
    +	 * @return An ActorRef to the TaskManager actor.
    +	 * @throws org.apache.flink.configuration.IllegalConfigurationException     Thrown, if the given config contains illegal values.
    +	 * @throws java.io.IOException      Thrown, if any of the I/O components (such as buffer pools,
    +	 *                                       I/O manager, ...) cannot be properly started.
    +	 * @throws java.lang.Exception      Thrown is some other error occurs while parsing the configuration
    +	 *                                      or starting the TaskManager components.
    +	 */
    +	public static ActorRef startTaskManagerComponentsAndActor(
    +		Configuration configuration,
    +		ResourceID resourceID,
    +		ActorSystem actorSystem,
    +		String taskManagerHostname,
    +		String taskManagerActorName,
    +		LeaderRetrievalService leaderRetrievalService,
    +		boolean localTaskManagerCommunication,
    +		Class<? extends TaskManager> taskManagerClass) throws Exception {
    +
    +		Tuple4<TaskManagerConfiguration, NetworkEnvironmentConfiguration, InstanceConnectionInfo, MemoryType> tuple4
    +			= parseTaskManagerConfiguration(configuration, taskManagerHostname, localTaskManagerCommunication);
    +
    +		TaskManagerConfiguration taskManagerConfig = tuple4._1();
    +		NetworkEnvironmentConfiguration netConfig = tuple4._2();
    +		InstanceConnectionInfo connectionInfo = tuple4._3();
    +		MemoryType memType = tuple4._4();
    +
    +		// pre-start checks
    +		checkTempDirs(taskManagerConfig.tmpDirPaths());
    +
    +		ExecutionContext executionContext = ExecutionContexts$.MODULE$.fromExecutor(new ForkJoinPool());
    +
    +		// we start the network first, to make sure it can allocate its buffers first
    +		NetworkEnvironment network = new NetworkEnvironment(executionContext, taskManagerConfig.timeout(), netConfig);
    +
    +		// computing the amount of memory to use depends on how much memory is available
    +		// it strictly needs to happen AFTER the network stack has been initialized
    +
    +		// check if a value has been configured
    +		long configuredMemory = configuration.getLong(ConfigConstants.TASK_MANAGER_MEMORY_SIZE_KEY, -1L);
    +		checkConfigParameter(configuredMemory == -1 || configuredMemory > 0, configuredMemory,
    +			ConfigConstants.TASK_MANAGER_MEMORY_SIZE_KEY,
    +			"MemoryManager needs at least one MB of memory. " +
    +				"If you leave this config parameter empty, the system automatically " +
    +				"pick a fraction of the available memory.");
    +
    +		long memorySize;
    +		boolean preAllocateMemory = configuration.getBoolean(
    +			ConfigConstants.TASK_MANAGER_MEMORY_PRE_ALLOCATE_KEY,
    +			ConfigConstants.DEFAULT_TASK_MANAGER_MEMORY_PRE_ALLOCATE);
    +		if (configuredMemory > 0) {
    +			if (preAllocateMemory) {
    +				LOG.info("Using {} MB for managed memory." , configuredMemory);
    +			} else {
    +				LOG.info("Limiting managed memory to {} MB, memory will be allocated lazily." , configuredMemory);
    +			}
    +			memorySize = configuredMemory << 20; // megabytes to bytes
    +		} else {
    +			float fraction = configuration.getFloat(
    +				ConfigConstants.TASK_MANAGER_MEMORY_FRACTION_KEY,
    +				ConfigConstants.DEFAULT_MEMORY_MANAGER_MEMORY_FRACTION);
    +			checkConfigParameter(fraction > 0.0f && fraction < 1.0f, fraction,
    +				ConfigConstants.TASK_MANAGER_MEMORY_FRACTION_KEY,
    +				"MemoryManager fraction of the free memory must be between 0.0 and 1.0");
    +
    +			if (memType == MemoryType.HEAP) {
    +				long relativeMemSize = (long) (EnvironmentInformation.getSizeOfFreeHeapMemoryWithDefrag() * fraction);
    +				if (preAllocateMemory) {
    +					LOG.info("Using {} of the currently free heap space for managed heap memory ({} MB)." ,
    +						fraction , relativeMemSize >> 20);
    +				} else {
    +					LOG.info("Limiting managed memory to {} of the currently free heap space ({} MB), " +
    +						"memory will be allocated lazily." , fraction , relativeMemSize >> 20);
    +				}
    +				memorySize = relativeMemSize;
    +			} else if (memType == MemoryType.OFF_HEAP) {
    +				// The maximum heap memory has been adjusted according to the fraction
    +				long maxMemory = EnvironmentInformation.getMaxJvmHeapMemory();
    +				long directMemorySize = (long) (maxMemory / (1.0 - fraction) * fraction);
    +				if (preAllocateMemory) {
    +					LOG.info("Using {} of the maximum memory size for managed off-heap memory ({} MB)." ,
    +						fraction, directMemorySize >> 20);
    +				} else {
    +					LOG.info("Limiting managed memory to {} of the maximum memory size ({} MB)," +
    +						" memory will be allocated lazily.", fraction, directMemorySize >> 20);
    +				}
    +				memorySize = directMemorySize;
    +			} else {
    +				throw new RuntimeException("No supported memory type detected.");
    +			}
    +		}
    +
    +		// now start the memory manager
    +		MemoryManager memoryManager;
    +		try {
    +			memoryManager = new MemoryManager(
    +				memorySize,
    +				taskManagerConfig.numberOfSlots(),
    +				netConfig.networkBufferSize(),
    +				memType,
    +				preAllocateMemory);
    +		} catch (OutOfMemoryError e) {
    +			if (memType == MemoryType.HEAP) {
    +				throw new Exception("OutOfMemory error (" + e.getMessage() +
    +					") while allocating the TaskManager heap memory (" + memorySize + " bytes).", e);
    +			} else if (memType == MemoryType.OFF_HEAP) {
    +				throw new Exception("OutOfMemory error (" + e.getMessage() +
    +					") while allocating the TaskManager off-heap memory (" + memorySize +
    +					" bytes).Try increasing the maximum direct memory (-XX:MaxDirectMemorySize)", e);
    +			} else {
    +				throw e;
    +			}
    +		}
    +
    +		// start the I/O manager, it will create some temp directories.
    +		IOManager ioManager = new IOManagerAsync(taskManagerConfig.tmpDirPaths());
    +
    +		if (leaderRetrievalService == null){
    +			leaderRetrievalService = LeaderRetrievalUtils.createLeaderRetrievalService(configuration);
    +		}
    +
    +		// create the actor properties (which define the actor constructor parameters)
    +		Props tmProps = Props.create(
    +			taskManagerClass,
    +			taskManagerConfig,
    +			resourceID,
    +			connectionInfo,
    +			memoryManager,
    +			ioManager,
    +			network,
    +			taskManagerConfig.numberOfSlots(),
    +			leaderRetrievalService);
    +
    +		ActorRef taskManagerActorRef;
    +		if (taskManagerActorName != null && !taskManagerActorName.equals("")) {
    +			taskManagerActorRef = actorSystem.actorOf(tmProps, taskManagerActorName);
    +		} else {
    +			taskManagerActorRef = actorSystem.actorOf(tmProps);
    +		}
    +
    +		return taskManagerActorRef;
    +	}
    +
    +	// --------------------------------------------------------------------------
    +	//  Parsing and checking the TaskManager Configuration
    +	// --------------------------------------------------------------------------
    +
    +	/**
    +	 * Utility method to extract TaskManager config parameters from the configuration and to
    +	 * sanity check them.
    +	 *
    +	 * @param configuration                 The configuration.
    +	 * @param taskManagerHostname           The host name under which the TaskManager communicates.
    +	 * @param localTaskManagerCommunication True, to skip initializing the network stack.
    +	 *                                      Use only in cases where only one task manager runs.
    +	 * @return A tuple (TaskManagerConfiguration, network configuration,
    +	 * InstanceConnectionInfo, JobManager actor Akka URL).
    +	 */
    +	private static Tuple4<TaskManagerConfiguration, NetworkEnvironmentConfiguration, InstanceConnectionInfo, MemoryType>
    +		parseTaskManagerConfiguration(Configuration configuration, String taskManagerHostname, boolean localTaskManagerCommunication)
    +		throws Exception {
    +
    +		// ------- read values from the config and check them ---------
    +		//                      (a lot of them)
    +
    +		// ----> hosts / ports for communication and data exchange
    +
    +		int dataport = configuration.getInteger(ConfigConstants.TASK_MANAGER_DATA_PORT_KEY,
    +			ConfigConstants.DEFAULT_TASK_MANAGER_DATA_PORT);
    +		if (dataport == 0) {
    +			dataport = NetUtils.getAvailablePort();
    +		}
    +		checkConfigParameter(dataport > 0, dataport, ConfigConstants.TASK_MANAGER_DATA_PORT_KEY,
    +			"Leave config parameter empty or use 0 to let the system choose a port automatically.");
    +
    +		InetAddress taskManagerAddress = InetAddress.getByName(taskManagerHostname);
    +		InstanceConnectionInfo connectionInfo = new InstanceConnectionInfo(taskManagerAddress, dataport);
    +
    +		// ----> memory / network stack (shuffles/broadcasts), task slots, temp directories
    +
    +		// we need this because many configs have been written with a "-1" entry
    +		int slots = configuration.getInteger(ConfigConstants.TASK_MANAGER_NUM_TASK_SLOTS, 1);
    +		if (slots == -1) {
    +			slots = 1;
    +		}
    +		checkConfigParameter(slots >= 1, slots, ConfigConstants.TASK_MANAGER_NUM_TASK_SLOTS,
    +			"Number of task slots must be at least one.");
    +
    +		int numNetworkBuffers = configuration.getInteger(
    +			ConfigConstants.TASK_MANAGER_NETWORK_NUM_BUFFERS_KEY,
    +			ConfigConstants.DEFAULT_TASK_MANAGER_NETWORK_NUM_BUFFERS);
    +		checkConfigParameter(numNetworkBuffers > 0, numNetworkBuffers,
    +			ConfigConstants.TASK_MANAGER_NETWORK_NUM_BUFFERS_KEY, "");
    +
    +		int pageSize = configuration.getInteger(
    +			ConfigConstants.TASK_MANAGER_MEMORY_SEGMENT_SIZE_KEY,
    +			ConfigConstants.DEFAULT_TASK_MANAGER_MEMORY_SEGMENT_SIZE);
    +		// check page size of for minimum size
    +		checkConfigParameter(pageSize >= MemoryManager.MIN_PAGE_SIZE, pageSize,
    +			ConfigConstants.TASK_MANAGER_MEMORY_SEGMENT_SIZE_KEY,
    +			"Minimum memory segment size is " + MemoryManager.MIN_PAGE_SIZE);
    +		// check page size for power of two
    +		checkConfigParameter(MathUtils.isPowerOf2(pageSize), pageSize,
    +			ConfigConstants.TASK_MANAGER_MEMORY_SEGMENT_SIZE_KEY,
    +			"Memory segment size must be a power of 2.");
    +
    +		// check whether we use heap or off-heap memory
    +		MemoryType memType;
    +		if (configuration.getBoolean(ConfigConstants.TASK_MANAGER_MEMORY_OFF_HEAP_KEY, false)) {
    +			memType = MemoryType.OFF_HEAP;
    +		} else {
    +			memType = MemoryType.HEAP;
    +		}
    +
    +		// initialize the memory segment factory accordingly
    +		if (memType == MemoryType.HEAP) {
    +			if (!MemorySegmentFactory.isInitialized()) {
    +				MemorySegmentFactory.initializeFactory(HeapMemorySegment.FACTORY);
    +			} else if (MemorySegmentFactory.getFactory() != HeapMemorySegment.FACTORY) {
    +				throw new Exception("Memory type is set to heap memory, but memory segment " +
    +					"factory has been initialized for off-heap memory segments");
    +			}
    +		} else {
    +			if (!MemorySegmentFactory.isInitialized()) {
    +				MemorySegmentFactory.initializeFactory(HybridMemorySegment.FACTORY);
    +			} else if (MemorySegmentFactory.getFactory() != HybridMemorySegment.FACTORY) {
    +				throw new Exception("Memory type is set to off-heap memory, but memory segment " +
    +					"factory has been initialized for heap memory segments");
    +			}
    +		}
    +
    +		String[] tmpDirs = configuration.getString(
    +			ConfigConstants.TASK_MANAGER_TMP_DIR_KEY,
    +			ConfigConstants.DEFAULT_TASK_MANAGER_TMP_PATH).split(",|" + File.pathSeparator);
    +
    +		NettyConfig nettyConfig = null;
    +		if (!localTaskManagerCommunication) {
    +			nettyConfig = new NettyConfig(connectionInfo.address(), connectionInfo.dataPort(), pageSize, slots, configuration);
    +		}
    +
    +		// Default spill I/O mode for intermediate results
    +		String syncOrAsync = configuration.getString(
    +			ConfigConstants.TASK_MANAGER_NETWORK_DEFAULT_IO_MODE,
    +			ConfigConstants.DEFAULT_TASK_MANAGER_NETWORK_DEFAULT_IO_MODE);
    +
    +		IOMode ioMode;
    +		if (syncOrAsync == "async") {
    +			ioMode = IOManager.IOMode.ASYNC;
    +		} else {
    +			ioMode = IOManager.IOMode.SYNC;
    +		}
    +
    +		NetworkEnvironmentConfiguration networkConfig = new NetworkEnvironmentConfiguration(
    +			numNetworkBuffers,
    +			pageSize,
    +			memType,
    +			ioMode,
    +			nettyConfig == null ? Option.<NettyConfig>empty() : new Some<>(nettyConfig),
    +			new Tuple2<>(500, 3000));
    +
    +		// ----> timeouts, library caching, profiling
    +
    +		FiniteDuration timeout;
    +		try {
    +			timeout = AkkaUtils.getTimeout(configuration);
    +		} catch (Exception e) {
    +			throw new IllegalArgumentException(
    +				"Invalid format for '" + ConfigConstants.AKKA_ASK_TIMEOUT +
    +					"'.Use formats like '50 s' or '1 min' to specify the timeout.");
    +		}
    +		LOG.info("Messages between TaskManager and JobManager have a max timeout of " + timeout);
    +
    +		long cleanupInterval = configuration.getLong(
    +			ConfigConstants.LIBRARY_CACHE_MANAGER_CLEANUP_INTERVAL,
    +			ConfigConstants.DEFAULT_LIBRARY_CACHE_MANAGER_CLEANUP_INTERVAL) * 1000;
    +
    +		FiniteDuration finiteRegistrationDuration = null;
    +		try {
    +			Duration maxRegistrationDuration = Duration.create(configuration.getString(
    +				ConfigConstants.TASK_MANAGER_MAX_REGISTRATION_DURATION,
    +				ConfigConstants.DEFAULT_TASK_MANAGER_MAX_REGISTRATION_DURATION));
    +			if (maxRegistrationDuration.isFinite()) {
    +				finiteRegistrationDuration = new FiniteDuration(maxRegistrationDuration.toSeconds(), TimeUnit.SECONDS);
    +			}
    +		} catch (NumberFormatException e) {
    +			throw new IllegalArgumentException("Invalid format for parameter " +
    +				ConfigConstants.TASK_MANAGER_MAX_REGISTRATION_DURATION, e);
    +		}
    +
    +		FiniteDuration initialRegistrationPause;
    +		try {
    +			Duration pause = Duration.create(configuration.getString(
    +				ConfigConstants.TASK_MANAGER_INITIAL_REGISTRATION_PAUSE,
    +				ConfigConstants.DEFAULT_TASK_MANAGER_INITIAL_REGISTRATION_PAUSE));
    +			if (pause.isFinite()) {
    +				initialRegistrationPause = new FiniteDuration(pause.toSeconds(), TimeUnit.SECONDS);
    +			} else {
    +				throw new IllegalArgumentException("The initial registration pause must be finite: " + pause);
    +			}
    +		} catch (NumberFormatException e) {
    +			throw new IllegalArgumentException("Invalid format for parameter " +
    +				ConfigConstants.TASK_MANAGER_INITIAL_REGISTRATION_PAUSE, e);
    +		}
    +
    +		FiniteDuration maxRegistrationPause;
    +		try {
    +			Duration pause = Duration.create(configuration.getString(
    +				ConfigConstants.TASK_MANAGER_MAX_REGISTARTION_PAUSE,
    +				ConfigConstants.DEFAULT_TASK_MANAGER_MAX_REGISTRATION_PAUSE));
    +			if (pause.isFinite()) {
    +				maxRegistrationPause = new FiniteDuration(pause.toSeconds(), TimeUnit.SECONDS);
    +			} else {
    +				throw new IllegalArgumentException("The maximum registration pause must be finite: " + pause);
    +			}
    +		} catch (NumberFormatException e) {
    +			throw new IllegalArgumentException("Invalid format for parameter " +
    +				ConfigConstants.TASK_MANAGER_INITIAL_REGISTRATION_PAUSE, e);
    +		}
    +
    +		FiniteDuration refusedRegistrationPause;
    +		try {
    +			Duration pause = Duration.create(configuration.getString(
    +				ConfigConstants.TASK_MANAGER_REFUSED_REGISTRATION_PAUSE,
    +				ConfigConstants.DEFAULT_TASK_MANAGER_REFUSED_REGISTRATION_PAUSE));
    +			if (pause.isFinite()) {
    +				refusedRegistrationPause = new FiniteDuration(pause.toSeconds(), TimeUnit.SECONDS);
    +			} else {
    +				throw new IllegalArgumentException("The refused registration pause must be finite: " + pause);
    +			}
    +		} catch (NumberFormatException e) {
    +			throw new IllegalArgumentException("Invalid format for parameter " +
    +				ConfigConstants.TASK_MANAGER_INITIAL_REGISTRATION_PAUSE, e);
    +		}
    +
    +		TaskManagerConfiguration taskManagerConfig = new TaskManagerConfiguration(
    --- End diff --
    
    Could we also convert `TaskManagerConfiguration` to Java?


---
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.
---

[GitHub] flink issue #2400: [FLINK-4363] Implement TaskManager basic startup of all c...

Posted by wangzhijiang999 <gi...@git.apache.org>.
Github user wangzhijiang999 commented on the issue:

    https://github.com/apache/flink/pull/2400
  
    @mxm, I already push my changes and solve the conflicts based on the new flip-6 branch.


---
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.
---