Skip to content

Commit

Permalink
Allow binding to multiple addresses.
Browse files Browse the repository at this point in the history
For consistency and simplicity this means that es.network.host/bind_host/publish_host can all take arrays.

Yes, we are still limited to a single publish host, and only the "best" address from publish_host will be used,
I do not change that, but its alread the case today (even by default: _local_ !), and its just an internal
limitation we can later fix. Instead we just continue with the same logic of "magic selection of publish host",
and users can always be explicit, and weare just clear to the user about what is happening.

I did it this way, because network stuff is already overwhelmed, and i mean really overwhelmed with settings.
We need to make this easy on the user to do the right thing and not make that worse.

Examples:

bin/elasticsearch # defaults
```
[elasticsearch] [2015-10-06 01:15:28,671][WARN ][common.network           ] [Todd Arliss] publish host: [_local_] resolves to multiple addresses, auto-selecting {127.0.0.1} as single publish address
[elasticsearch] [2015-10-06 01:15:28,672][INFO ][transport                ] [Todd Arliss] publish_address {127.0.0.1:9300}, bound_addresses {[fe80::1]:9300}, {[::1]:9300}, {127.0.0.1:9300}
```

bin/elasticsearch -Des.network.host=_lo0_,_en0_ # multiple interfaces
```
[2015-10-06 01:17:08,970][WARN ][common.network           ] [Torso] publish host: [_lo0_, _en0_] resolves to multiple addresses, auto-selecting {192.168.0.19} as single publish address
[2015-10-06 01:17:08,972][INFO ][transport                ] [Torso] publish_address {192.168.0.19:9300}, bound_addresses {[fe80::1]:9300}, {[::1]:9300}, {127.0.0.1:9300}, {[fe80::3e15:c2ff:fee5:d26c]:9300}, {192.168.0.19:9300}
```

bin/elasticsearch -Des.network.host=192.168.0.19,_local_ -Des.network.publish_host=192.168.0.19 # being explicit
```
[2015-10-06 01:19:28,603][INFO ][transport                ] [Thunderbird] publish_address {192.168.0.19:9300}, bound_addresses {[fe80::1]:9300}, {[::1]:9300}, {127.0.0.1:9300}, {192.168.0.19:9300}
```

Closes elastic#13592
  • Loading branch information
rmuir committed Oct 6, 2015
1 parent f2fa279 commit 6bc77f5
Show file tree
Hide file tree
Showing 10 changed files with 134 additions and 81 deletions.
Expand Up @@ -28,6 +28,9 @@
import java.io.IOException;
import java.net.InetAddress;
import java.net.UnknownHostException;
import java.util.ArrayList;
import java.util.Arrays;
import java.util.HashSet;
import java.util.List;
import java.util.concurrent.CopyOnWriteArrayList;
import java.util.concurrent.TimeUnit;
Expand Down Expand Up @@ -91,13 +94,21 @@ public void addCustomNameResolver(CustomNameResolver customNameResolver) {
customNameResolvers.add(customNameResolver);
}

public InetAddress[] resolveBindHostAddress(String bindHost) throws IOException {
/**
* Resolves {@code bindHosts} to a list of internet addresses. The list will
* not contain duplicate addresses.
* @param bindHosts list of hosts to bind to. this may contain special pseudo-hostnames
* such as _local_ (see the documentation). if it is null, it will be populated
* based on global default settings.
* @return unique set of internet addresses
*/
public InetAddress[] resolveBindHostAddresses(String bindHosts[]) throws IOException {
// first check settings
if (bindHost == null) {
bindHost = settings.get(GLOBAL_NETWORK_BINDHOST_SETTING, settings.get(GLOBAL_NETWORK_HOST_SETTING));
if (bindHosts == null) {
bindHosts = settings.getAsArray(GLOBAL_NETWORK_BINDHOST_SETTING, settings.getAsArray(GLOBAL_NETWORK_HOST_SETTING, null));
}
// next check any registered custom resolvers
if (bindHost == null) {
if (bindHosts == null) {
for (CustomNameResolver customNameResolver : customNameResolvers) {
InetAddress addresses[] = customNameResolver.resolveDefault();
if (addresses != null) {
Expand All @@ -106,31 +117,44 @@ public InetAddress[] resolveBindHostAddress(String bindHost) throws IOException
}
}
// finally, fill with our default
if (bindHost == null) {
bindHost = DEFAULT_NETWORK_HOST;
if (bindHosts == null) {
bindHosts = new String[] { DEFAULT_NETWORK_HOST };
}
InetAddress addresses[] = resolveInetAddress(bindHost);
InetAddress addresses[] = resolveInetAddresses(bindHosts);

// try to deal with some (mis)configuration
if (addresses != null) {
for (InetAddress address : addresses) {
// check if its multicast: flat out mistake
if (address.isMulticastAddress()) {
throw new IllegalArgumentException("bind address: {" + NetworkAddress.format(address) + "} is invalid: multicast address");
}
for (InetAddress address : addresses) {
// check if its multicast: flat out mistake
if (address.isMulticastAddress()) {
throw new IllegalArgumentException("bind address: {" + NetworkAddress.format(address) + "} is invalid: multicast address");
}
// check if its a wildcard address: this is only ok if its the only address!
if (address.isAnyLocalAddress() && addresses.length > 1) {
throw new IllegalArgumentException("bind address: {" + NetworkAddress.format(address) + "} is wildcard, but multiple addresses specified: this makes no sense");
}
}
return addresses;
}

/**
* Resolves {@code publishHosts} to a single publish address. The fact that it returns
* only one address is just a current limitation.
* <p>
* If {@code publishHosts} resolves to more than one address, <b>then one is selected with magic</b>,
* and the user is warned (they can always just be more specific).
* @param publishHosts list of hosts to publish as. this may contain special pseudo-hostnames
* such as _local_ (see the documentation). if it is null, it will be populated
* based on global default settings.
* @return single internet address
*/
// TODO: needs to be InetAddress[]
public InetAddress resolvePublishHostAddress(String publishHost) throws IOException {
public InetAddress resolvePublishHostAddresses(String publishHosts[]) throws IOException {
// first check settings
if (publishHost == null) {
publishHost = settings.get(GLOBAL_NETWORK_PUBLISHHOST_SETTING, settings.get(GLOBAL_NETWORK_HOST_SETTING));
if (publishHosts == null) {
publishHosts = settings.getAsArray(GLOBAL_NETWORK_PUBLISHHOST_SETTING, settings.getAsArray(GLOBAL_NETWORK_HOST_SETTING, null));
}
// next check any registered custom resolvers
if (publishHost == null) {
if (publishHosts == null) {
for (CustomNameResolver customNameResolver : customNameResolvers) {
InetAddress addresses[] = customNameResolver.resolveDefault();
if (addresses != null) {
Expand All @@ -139,30 +163,61 @@ public InetAddress resolvePublishHostAddress(String publishHost) throws IOExcept
}
}
// finally, fill with our default
if (publishHost == null) {
publishHost = DEFAULT_NETWORK_HOST;
if (publishHosts == null) {
publishHosts = new String[] { DEFAULT_NETWORK_HOST };
}
InetAddress addresses[] = resolveInetAddresses(publishHosts);
// TODO: allow publishing multiple addresses
InetAddress address = resolveInetAddress(publishHost)[0];
// for now... the hack begins

// try to deal with some (mis)configuration
if (address != null) {
// 1. single wildcard address, probably set by network.host
if (addresses.length == 1 && addresses[0].isAnyLocalAddress()) {
InetAddress old = addresses[0];
addresses = resolveInetAddresses(new String[] { "_non_loopback_" });
logger.warn("publish address: {{}} is a wildcard address, falling back to _non_loopback_",
NetworkAddress.format(old));
}

// 2. try to deal with some (mis)configuration
for (InetAddress address : addresses) {
// check if its multicast: flat out mistake
if (address.isMulticastAddress()) {
throw new IllegalArgumentException("publish address: {" + NetworkAddress.format(address) + "} is invalid: multicast address");
}
// wildcard address, probably set by network.host
// check if its a wildcard address: this is only ok if its the only address!
// (if it was a single wildcard address, it was replaced by step 1 above)
if (address.isAnyLocalAddress()) {
InetAddress old = address;
address = NetworkUtils.getFirstNonLoopbackAddresses()[0];
logger.warn("publish address: {{}} is a wildcard address, falling back to first non-loopback: {{}}",
NetworkAddress.format(old), NetworkAddress.format(address));
throw new IllegalArgumentException("publish address: {" + NetworkAddress.format(address) + "} is wildcard, but multiple addresses specified: this makes no sense");
}
}
return address;

// 3. warn user if we end out with multiple publish addresses
if (addresses.length > 1) {
List<InetAddress> sorted = new ArrayList<>(Arrays.asList(addresses));
NetworkUtils.sortAddresses(sorted);
addresses = new InetAddress[] { sorted.get(0) };
logger.warn("publish host: {} resolves to multiple addresses, auto-selecting {{}} as single publish address",
Arrays.toString(publishHosts), NetworkAddress.format(addresses[0]));
}
return addresses[0];
}

/** resolves (and deduplicates) host specification */
private InetAddress[] resolveInetAddresses(String hosts[]) throws UnknownHostException, IOException {
if (hosts.length == 0) {
throw new IllegalArgumentException("empty host specification");
}
// deduplicate, in case of resolver misconfiguration
// stuff like https://bugzilla.redhat.com/show_bug.cgi?id=496300
HashSet<InetAddress> set = new HashSet<>();
for (String host : hosts) {
set.addAll(Arrays.asList(resolveInternal(host)));
}
return set.toArray(new InetAddress[set.size()]);
}

private InetAddress[] resolveInetAddress(String host) throws UnknownHostException, IOException {
/** resolves a single host specification */
private InetAddress[] resolveInternal(String host) throws UnknownHostException, IOException {
if ((host.startsWith("#") && host.endsWith("#")) || (host.startsWith("_") && host.endsWith("_"))) {
host = host.substring(1, host.length() - 1);
// allow custom resolvers to have special names
Expand Down Expand Up @@ -198,6 +253,6 @@ private InetAddress[] resolveInetAddress(String host) throws UnknownHostExceptio
}
}
}
return NetworkUtils.getAllByName(host);
return InetAddress.getAllByName(host);
}
}
Expand Up @@ -27,12 +27,10 @@
import java.net.InetAddress;
import java.net.NetworkInterface;
import java.net.SocketException;
import java.net.UnknownHostException;
import java.util.ArrayList;
import java.util.Arrays;
import java.util.Collections;
import java.util.Comparator;
import java.util.HashSet;
import java.util.List;

/**
Expand Down Expand Up @@ -109,7 +107,8 @@ static int sortKey(InetAddress address, boolean prefer_v6) {
* @deprecated remove this when multihoming is really correct
*/
@Deprecated
static void sortAddresses(List<InetAddress> list) {
// only public because of silly multicast
public static void sortAddresses(List<InetAddress> list) {
Collections.sort(list, new Comparator<InetAddress>() {
@Override
public int compare(InetAddress left, InetAddress right) {
Expand Down Expand Up @@ -161,7 +160,6 @@ static InetAddress[] getLoopbackAddresses() throws SocketException {
if (list.isEmpty()) {
throw new IllegalArgumentException("No up-and-running loopback interfaces found, got " + getInterfaces());
}
sortAddresses(list);
return list.toArray(new InetAddress[list.size()]);
}

Expand All @@ -177,7 +175,6 @@ static InetAddress[] getFirstNonLoopbackAddresses() throws SocketException {
if (list.isEmpty()) {
throw new IllegalArgumentException("No up-and-running non-loopback interfaces found, got " + getInterfaces());
}
sortAddresses(list);
return list.toArray(new InetAddress[list.size()]);
}

Expand All @@ -194,20 +191,9 @@ static InetAddress[] getAddressesForInterface(String name) throws SocketExceptio
if (list.isEmpty()) {
throw new IllegalArgumentException("Interface '" + name + "' has no internet addresses");
}
sortAddresses(list);
return list.toArray(new InetAddress[list.size()]);
}

/** Returns addresses for the given host, sorted by order of preference */
static InetAddress[] getAllByName(String host) throws UnknownHostException {
InetAddress addresses[] = InetAddress.getAllByName(host);
// deduplicate, in case of resolver misconfiguration
// stuff like https://bugzilla.redhat.com/show_bug.cgi?id=496300
List<InetAddress> unique = new ArrayList<>(new HashSet<>(Arrays.asList(addresses)));
sortAddresses(unique);
return unique.toArray(new InetAddress[unique.size()]);
}

/** Returns only the IPV4 addresses in {@code addresses} */
static InetAddress[] filterIPV4(InetAddress addresses[]) {
List<InetAddress> list = new ArrayList<>();
Expand Down
Expand Up @@ -50,6 +50,7 @@
import java.net.InetAddress;
import java.net.InetSocketAddress;
import java.util.ArrayList;
import java.util.Arrays;
import java.util.List;
import java.util.concurrent.Executors;
import java.util.concurrent.atomic.AtomicReference;
Expand Down Expand Up @@ -105,9 +106,9 @@ public class NettyHttpServerTransport extends AbstractLifecycleComponent<HttpSer

protected final String port;

protected final String bindHost;
protected final String bindHosts[];

protected final String publishHost;
protected final String publishHosts[];

protected final boolean detailedErrorsEnabled;

Expand Down Expand Up @@ -157,8 +158,8 @@ public NettyHttpServerTransport(Settings settings, NetworkService networkService
this.workerCount = settings.getAsInt("http.netty.worker_count", EsExecutors.boundedNumberOfProcessors(settings) * 2);
this.blockingServer = settings.getAsBoolean("http.netty.http.blocking_server", settings.getAsBoolean(TCP_BLOCKING_SERVER, settings.getAsBoolean(TCP_BLOCKING, false)));
this.port = settings.get("http.netty.port", settings.get("http.port", "9200-9300"));
this.bindHost = settings.get("http.netty.bind_host", settings.get("http.bind_host", settings.get("http.host")));
this.publishHost = settings.get("http.netty.publish_host", settings.get("http.publish_host", settings.get("http.host")));
this.bindHosts = settings.getAsArray("http.netty.bind_host", settings.getAsArray("http.bind_host", settings.getAsArray("http.host", null)));
this.publishHosts = settings.getAsArray("http.netty.publish_host", settings.getAsArray("http.publish_host", settings.getAsArray("http.host", null)));
this.publishPort = settings.getAsInt("http.netty.publish_port", settings.getAsInt("http.publish_port", 0));
this.tcpNoDelay = settings.get("http.netty.tcp_no_delay", settings.get(TCP_NO_DELAY, "true"));
this.tcpKeepAlive = settings.get("http.netty.tcp_keep_alive", settings.get(TCP_KEEP_ALIVE, "true"));
Expand Down Expand Up @@ -246,9 +247,9 @@ protected void doStart() {
// Bind and start to accept incoming connections.
InetAddress hostAddresses[];
try {
hostAddresses = networkService.resolveBindHostAddress(bindHost);
hostAddresses = networkService.resolveBindHostAddresses(bindHosts);
} catch (IOException e) {
throw new BindHttpException("Failed to resolve host [" + bindHost + "]", e);
throw new BindHttpException("Failed to resolve host [" + Arrays.toString(bindHosts) + "]", e);
}

List<InetSocketTransportAddress> boundAddresses = new ArrayList<>(hostAddresses.length);
Expand All @@ -262,7 +263,7 @@ protected void doStart() {
publishPort = boundAddress.getPort();
}
try {
publishAddress = new InetSocketAddress(networkService.resolvePublishHostAddress(publishHost), publishPort);
publishAddress = new InetSocketAddress(networkService.resolvePublishHostAddresses(publishHosts), publishPort);
} catch (Exception e) {
throw new BindTransportException("Failed to resolve publish address", e);
}
Expand Down
Expand Up @@ -343,9 +343,9 @@ public Map<String, BoundTransportAddress> profileBoundAddresses() {
return ImmutableMap.copyOf(profileBoundAddresses);
}

private InetSocketAddress createPublishAddress(String publishHost, int publishPort) {
private InetSocketAddress createPublishAddress(String publishHosts[], int publishPort) {
try {
return new InetSocketAddress(networkService.resolvePublishHostAddress(publishHost), publishPort);
return new InetSocketAddress(networkService.resolvePublishHostAddresses(publishHosts), publishPort);
} catch (Exception e) {
throw new BindTransportException("Failed to resolve publish address", e);
}
Expand Down Expand Up @@ -436,11 +436,11 @@ private Settings createFallbackSettings() {
private void bindServerBootstrap(final String name, final Settings settings) {
// Bind and start to accept incoming connections.
InetAddress hostAddresses[];
String bindHost = settings.get("bind_host");
String bindHosts[] = settings.getAsArray("bind_host", null);
try {
hostAddresses = networkService.resolveBindHostAddress(bindHost);
hostAddresses = networkService.resolveBindHostAddresses(bindHosts);
} catch (IOException e) {
throw new BindTransportException("Failed to resolve host [" + bindHost + "]", e);
throw new BindTransportException("Failed to resolve host " + Arrays.toString(bindHosts) + "", e);
}
if (logger.isDebugEnabled()) {
String[] addresses = new String[hostAddresses.length];
Expand Down Expand Up @@ -493,8 +493,8 @@ public boolean onPortNumber(int portNumber) {
if (boundTransportAddress == null) {
// no address is bound, so lets create one with the publish address information from the settings or the bound address as a fallback
int publishPort = profileSettings.getAsInt("publish_port", boundAddress.getPort());
String publishHost = profileSettings.get("publish_host", boundAddress.getHostString());
InetSocketAddress publishAddress = createPublishAddress(publishHost, publishPort);
String publishHosts[] = profileSettings.getAsArray("publish_host", new String[] { boundAddress.getHostString() });
InetSocketAddress publishAddress = createPublishAddress(publishHosts, publishPort);
profileBoundAddresses.put(name, new BoundTransportAddress(new TransportAddress[]{new InetSocketTransportAddress(boundAddress)}, new InetSocketTransportAddress(publishAddress)));
} else {
// TODO: support real multihoming with publishing. Today we update the bound addresses so only the prioritized address is published
Expand All @@ -511,8 +511,8 @@ public boolean onPortNumber(int portNumber) {
// these calls are different from the profile ones due to the way the settings for a profile are created. If we want to merge the code for the default profile and
// other profiles together, we need to change how the profileSettings are built for the default profile...
int publishPort = settings.getAsInt("transport.netty.publish_port", settings.getAsInt("transport.publish_port", boundAddress.getPort()));
String publishHost = settings.get("transport.netty.publish_host", settings.get("transport.publish_host", settings.get("transport.host")));
InetSocketAddress publishAddress = createPublishAddress(publishHost, publishPort);
String publishHosts[] = settings.getAsArray("transport.netty.publish_host", settings.getAsArray("transport.publish_host", settings.getAsArray("transport.host", null)));
InetSocketAddress publishAddress = createPublishAddress(publishHosts, publishPort);
this.boundAddress = new BoundTransportAddress(new TransportAddress[]{new InetSocketTransportAddress(boundAddress)}, new InetSocketTransportAddress(publishAddress));
} else {
// the default profile is already bound to one address and has the publish address, copy the existing bound addresses as is and append the new address.
Expand Down

0 comments on commit 6bc77f5

Please sign in to comment.