Skip to content

Commit

Permalink
[FLINK-2967] Make delay for local host connection strategy configurable
Browse files Browse the repository at this point in the history
  • Loading branch information
rmetzger committed Nov 4, 2015
1 parent 81f8c05 commit 63b9b3c
Show file tree
Hide file tree
Showing 9 changed files with 59 additions and 23 deletions.
Original file line number Diff line number Diff line change
Expand Up @@ -36,17 +36,18 @@ public final class DataStreamUtils {
* @return The iterator
*/
public static <OUT> Iterator<OUT> collect(DataStream<OUT> stream) {
TypeSerializer serializer = stream.getType().createSerializer(stream.getExecutionEnvironment().getConfig());
DataStreamIterator<OUT> it = new DataStreamIterator<OUT>(serializer);
TypeSerializer<OUT> serializer = stream.getType().createSerializer(stream.getExecutionEnvironment().getConfig());
DataStreamIterator<OUT> it = new DataStreamIterator<>(serializer);

//Find out what IP of us should be given to CollectSink, that it will be able to connect to
StreamExecutionEnvironment env = stream.getExecutionEnvironment();
InetAddress clientAddress;
if(env instanceof RemoteStreamEnvironment) {
String host = ((RemoteStreamEnvironment)env).getHost();
int port = ((RemoteStreamEnvironment)env).getPort();
RemoteStreamEnvironment rse = (RemoteStreamEnvironment) env;
String host = rse.getHost();
int port = rse.getPort();
try {
clientAddress = ConnectionUtils.findConnectingAddress(new InetSocketAddress(host, port), 2000, 400);
clientAddress = ConnectionUtils.findConnectingAddress(new InetSocketAddress(host, port), 2000, 400, rse.getConfiguration());
} catch (IOException e) {
throw new RuntimeException("IOException while trying to connect to the master", e);
}
Expand All @@ -61,7 +62,7 @@ public static <OUT> Iterator<OUT> collect(DataStream<OUT> stream) {
DataStreamSink<OUT> sink = stream.addSink(new CollectSink<OUT>(clientAddress, it.getPort(), serializer));
sink.setParallelism(1); // It would not work if multiple instances would connect to the same port

(new CallExecute(stream)).start();
(new CallExecute<>(stream)).start();

return it;
}
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -160,6 +160,16 @@ public final class ConfigConstants {
*/
public static final String TASK_MANAGER_MAX_REGISTRATION_DURATION = "taskmanager.maxRegistrationDuration";

/**
* The first strategy of the TaskManager to detect its own (externally reachable) IP address
* is using the address returned by InetAddress.getLocalHost().
* The default timeout for checking that IP address is 50 ms. This configuration parameter allows
* users to set a higher value.
*
* See FLINK-2967 for details.
*/
public static final String TASK_MANAGER_ADDRESS_DETECTION_LOCAL_HOST_TIMEOUT = "taskmanager.address-detection.local-host-timeout";

// --------------------------- Runtime Algorithms -------------------------------

/**
Expand Down Expand Up @@ -665,7 +675,8 @@ public final class ConfigConstants {
* The default path to the file containing the list of access privileged users and passwords.
*/
public static final String DEFAULT_WEB_ACCESS_FILE_PATH = null;



// ------------------------------ Akka Values ------------------------------

public static String DEFAULT_AKKA_TRANSPORT_HEARTBEAT_INTERVAL = "1000 s";
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -29,6 +29,8 @@
import java.util.UUID;
import java.util.concurrent.TimeUnit;

import org.apache.flink.configuration.ConfigConstants;
import org.apache.flink.configuration.Configuration;
import org.apache.flink.runtime.akka.AkkaUtils;
import org.apache.flink.runtime.leaderretrieval.LeaderRetrievalException;
import org.apache.flink.runtime.leaderretrieval.LeaderRetrievalListener;
Expand Down Expand Up @@ -93,9 +95,10 @@ public int getTimeout() {
* @param maxWaitMillis The maximum time that this method tries to connect, before falling
* back to the heuristics.
* @param startLoggingAfter The time after which the method will log on INFO level.
* @param conf Configuration object for the address detecton
*/
public static InetAddress findConnectingAddress(InetSocketAddress targetAddress,
long maxWaitMillis, long startLoggingAfter) throws IOException
long maxWaitMillis, long startLoggingAfter, Configuration conf) throws IOException
{
if (targetAddress == null) {
throw new NullPointerException("targetAddress must not be null");
Expand All @@ -119,7 +122,7 @@ public static InetAddress findConnectingAddress(InetSocketAddress targetAddress,
}
// go over the strategies ADDRESS - FAST_CONNECT - SLOW_CONNECT
do {
InetAddress address = findAddressUsingStrategy(strategy, targetAddress, logging);
InetAddress address = findAddressUsingStrategy(strategy, targetAddress, logging, conf);
if (address != null) {
return address;
}
Expand Down Expand Up @@ -170,7 +173,7 @@ public static InetAddress findConnectingAddress(InetSocketAddress targetAddress,

// our attempts timed out. use the heuristic fallback
LOG.warn("Could not connect to {}. Selecting a local address using heuristics.", targetAddress);
InetAddress heuristic = findAddressUsingStrategy(AddressDetectionState.HEURISTIC, targetAddress, true);
InetAddress heuristic = findAddressUsingStrategy(AddressDetectionState.HEURISTIC, targetAddress, true, conf);
if (heuristic != null) {
return heuristic;
}
Expand All @@ -183,13 +186,19 @@ public static InetAddress findConnectingAddress(InetSocketAddress targetAddress,

private static InetAddress findAddressUsingStrategy(AddressDetectionState strategy,
InetSocketAddress targetAddress,
boolean logging) throws IOException
boolean logging,
Configuration conf) throws IOException
{
// try LOCAL_HOST strategy independent of the network interfaces
if(strategy == AddressDetectionState.LOCAL_HOST) {
InetAddress localhostName = InetAddress.getLocalHost();

if(tryToConnect(localhostName, targetAddress, strategy.getTimeout(), logging)) {
int timeout = strategy.getTimeout();
// check for a custom timeout
if(conf.getInteger(ConfigConstants.TASK_MANAGER_ADDRESS_DETECTION_LOCAL_HOST_TIMEOUT, -1) != -1 ) {
timeout = conf.getInteger(ConfigConstants.TASK_MANAGER_ADDRESS_DETECTION_LOCAL_HOST_TIMEOUT, -1);
LOG.info("Using the custom-configured timeout for the LOCAL_HOST address detection strategy: {}", timeout);
}
if(tryToConnect(localhostName, targetAddress, timeout, logging)) {
LOG.debug("Using InetAddress.getLocalHost() immediately for the connecting address");
return localhostName;
} else {
Expand Down Expand Up @@ -315,6 +324,11 @@ private enum LeaderRetrievalState {
private String akkaURL;
private LeaderRetrievalState retrievalState = LeaderRetrievalState.NOT_RETRIEVED;
private Exception exception;
private Configuration configuration;

public LeaderConnectingAddressListener(Configuration configuration) {
this.configuration = configuration;
}

public InetAddress findConnectingAddress(
FiniteDuration timeout) throws LeaderRetrievalException {
Expand Down Expand Up @@ -369,7 +383,7 @@ public InetAddress findConnectingAddress(
}

do {
InetAddress address = ConnectionUtils.findAddressUsingStrategy(strategy, targetAddress, logging);
InetAddress address = ConnectionUtils.findAddressUsingStrategy(strategy, targetAddress, logging, configuration);
if (address != null) {
return address;
}
Expand Down Expand Up @@ -418,7 +432,7 @@ public InetAddress findConnectingAddress(

if (targetAddress != null) {
LOG.warn("Could not connect to {}. Selecting a local address using heuristics.", targetAddress);
heuristic = findAddressUsingStrategy(AddressDetectionState.HEURISTIC, targetAddress, true);
heuristic = findAddressUsingStrategy(AddressDetectionState.HEURISTIC, targetAddress, true, configuration);
}

if (heuristic != null) {
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -146,8 +146,8 @@ public static LeaderConnectionInfo retrieveLeaderConnectionInfo(

public static InetAddress findConnectingAddress(
LeaderRetrievalService leaderRetrievalService,
FiniteDuration timeout) throws LeaderRetrievalException {
ConnectionUtils.LeaderConnectingAddressListener listener = new ConnectionUtils.LeaderConnectingAddressListener();
FiniteDuration timeout, Configuration configuration) throws LeaderRetrievalException {
ConnectionUtils.LeaderConnectingAddressListener listener = new ConnectionUtils.LeaderConnectingAddressListener(configuration);

try {
leaderRetrievalService.start(listener);
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -1353,7 +1353,8 @@ object TaskManager {

val taskManagerAddress = LeaderRetrievalUtils.findConnectingAddress(
leaderRetrievalService,
lookupTimeout)
lookupTimeout,
configuration)

taskManagerHostname = taskManagerAddress.getHostName()
LOG.info(s"TaskManager will use hostname/address '${taskManagerHostname}' " +
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -185,7 +185,7 @@ public void testTimeoutOfFindConnectingAddress() throws Exception {
FiniteDuration timeout = new FiniteDuration(10, TimeUnit.SECONDS);

LeaderRetrievalService leaderRetrievalService = LeaderRetrievalUtils.createLeaderRetrievalService(config);
InetAddress result = LeaderRetrievalUtils.findConnectingAddress(leaderRetrievalService, timeout);
InetAddress result = LeaderRetrievalUtils.findConnectingAddress(leaderRetrievalService, timeout, config);

assertEquals(InetAddress.getLocalHost(), result);
}
Expand All @@ -207,7 +207,7 @@ public FindConnectingAddress(Configuration config, FiniteDuration timeout) {
public void run() {
try {
LeaderRetrievalService leaderRetrievalService = LeaderRetrievalUtils.createLeaderRetrievalService(config);
result = LeaderRetrievalUtils.findConnectingAddress(leaderRetrievalService, timeout);
result = LeaderRetrievalUtils.findConnectingAddress(leaderRetrievalService, timeout, this.config);
} catch (Exception e) {
exception = e;
}
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -19,6 +19,7 @@

import static org.junit.Assert.*;

import org.apache.flink.configuration.Configuration;
import org.apache.flink.util.OperatingSystem;
import org.junit.Test;

Expand Down Expand Up @@ -47,7 +48,7 @@ public void testFindConnectableAddress() {
InetSocketAddress unreachable = new InetSocketAddress("localhost", unusedPort);

final long start = System.currentTimeMillis();
InetAddress add = ConnectionUtils.findConnectingAddress(unreachable, 2000, 400);
InetAddress add = ConnectionUtils.findConnectingAddress(unreachable, 2000, 400, new Configuration());

// check that it did not take forever
assertTrue(System.currentTimeMillis() - start < (OperatingSystem.isWindows() ? 30000 : 8000));
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -250,4 +250,12 @@ public String getHost() {
public int getPort() {
return port;
}

/**
* Return Flink configuration
* @return Flink configuration
*/
public Configuration getConfiguration() {
return config;
}
}
Original file line number Diff line number Diff line change
Expand Up @@ -147,9 +147,9 @@ public void connectToCluster() throws IOException {
LOG.info("Start actor system.");
// find name of own public interface, able to connect to the JM
// try to find address for 2 seconds. log after 400 ms.
InetAddress ownHostname = ConnectionUtils.findConnectingAddress(jobManagerAddress, 2000, 400);
InetAddress ownHostname = ConnectionUtils.findConnectingAddress(jobManagerAddress, 2000, 400, this.flinkConfig);
actorSystem = AkkaUtils.createActorSystem(flinkConfig,
new Some(new Tuple2<String, Integer>(ownHostname.getCanonicalHostName(), 0)));
new Some(new Tuple2<>(ownHostname.getCanonicalHostName(), 0)));

// Create the leader election service
flinkConfig.setString(ConfigConstants.JOB_MANAGER_IPC_ADDRESS_KEY, this.jobManagerAddress.getHostName());
Expand Down

0 comments on commit 63b9b3c

Please sign in to comment.