Skip to content

Reject port ranges in discovery.seed_hosts #41905

New issue

Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.

By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.

Already on GitHub? Sign in to your account

Merged
merged 1 commit into from
May 8, 2019
Merged
Show file tree
Hide file tree
Changes from all commits
Commits
File filter

Filter by extension

Filter by extension

Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
12 changes: 12 additions & 0 deletions docs/reference/migration/migrate_7_2.asciidoc
Original file line number Diff line number Diff line change
Expand Up @@ -17,3 +17,15 @@ coming[7.2.0]
//tag::notable-breaking-changes[]

// end::notable-breaking-changes[]

[[breaking_72_discovery_changes]]
=== Discovery changes

[float]
==== Only a single port may be given for each seed host.

In earlier versions you could include a range of ports in entries in the
`discovery.seed_hosts` list, but {es} used only the first port in the range and
unexpectedly ignored the rest. For instance if you set `discovery.seed_hosts:
"10.11.12.13:9300-9310"` then {es} would only use `10.11.12.13:9300` for
discovery. Seed host addresses containing port ranges are now rejected.
Original file line number Diff line number Diff line change
Expand Up @@ -208,8 +208,7 @@ public List<TransportAddress> getSeedAddresses(HostsResolver hostsResolver) {
}

try {
// we only limit to 1 port per address, makes no sense to ping 100 ports
TransportAddress[] addresses = transportService.addressesFromString(networkAddress, 1);
TransportAddress[] addresses = transportService.addressesFromString(networkAddress);
for (TransportAddress address : addresses) {
logger.trace("adding {}, transport_address {}", networkAddress, address);
dynamicHosts.add(address);
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -174,8 +174,7 @@ && disjoint(securityGroupIds, groups)) {
}
if (address != null) {
try {
// we only limit to 1 port per address, makes no sense to ping 100 ports
final TransportAddress[] addresses = transportService.addressesFromString(address, 1);
final TransportAddress[] addresses = transportService.addressesFromString(address);
for (int i = 0; i < addresses.length; i++) {
logger.trace("adding {}, address {}, transport_address {}", instance.getInstanceId(), address, addresses[i]);
dynamicHosts.add(addresses[i]);
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -77,7 +77,7 @@ public void createTransportService() {
new NetworkService(Collections.emptyList()), PageCacheRecycler.NON_RECYCLING_INSTANCE, namedWriteableRegistry,
new NoneCircuitBreakerService()) {
@Override
public TransportAddress[] addressesFromString(String address, int perAddressLimit) throws UnknownHostException {
public TransportAddress[] addressesFromString(String address) throws UnknownHostException {
// we just need to ensure we don't resolve DNS here
return new TransportAddress[] {poorMansDNS.getOrDefault(address, buildNewFakeTransportAddress())};
}
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -233,8 +233,7 @@ public List<TransportAddress> getSeedAddresses(HostsResolver hostsResolver) {

// ip_private is a single IP Address. We need to build a TransportAddress from it
// If user has set `es_port` metadata, we don't need to ping all ports
// we only limit to 1 addresses, makes no sense to ping 100 ports
TransportAddress[] addresses = transportService.addressesFromString(address, 1);
TransportAddress[] addresses = transportService.addressesFromString(address);

for (TransportAddress transportAddress : addresses) {
logger.trace("adding {}, type {}, address {}, transport_address {}, status {}", name, type,
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -75,7 +75,7 @@ private List<String> getHostsList() {

@Override
public List<TransportAddress> getSeedAddresses(HostsResolver hostsResolver) {
final List<TransportAddress> transportAddresses = hostsResolver.resolveHosts(getHostsList(), 1);
final List<TransportAddress> transportAddresses = hostsResolver.resolveHosts(getHostsList());
logger.debug("seed addresses: {}", transportAddresses);
return transportAddresses;
}
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -36,10 +36,9 @@ public interface SeedHostsProvider {

/**
* Helper object that allows to resolve a list of hosts to a list of transport addresses.
* Each host is resolved into a transport address (or a collection of addresses if the
* number of ports is greater than one)
* Each host is resolved into a transport address
*/
interface HostsResolver {
List<TransportAddress> resolveHosts(List<String> hosts, int limitPortCounts);
List<TransportAddress> resolveHosts(List<String> hosts);
}
}
Original file line number Diff line number Diff line change
Expand Up @@ -116,7 +116,6 @@ public static TimeValue getResolveTimeout(Settings settings) {
* @param executorService the executor service used to parallelize hostname lookups
* @param logger logger used for logging messages regarding hostname lookups
* @param hosts the hosts to resolve
* @param limitPortCounts the number of ports to resolve (should be 1 for non-local transport)
* @param transportService the transport service
* @param resolveTimeout the timeout before returning from hostname lookups
* @return a list of resolved transport addresses
Expand All @@ -125,7 +124,6 @@ public static List<TransportAddress> resolveHostsLists(
final ExecutorService executorService,
final Logger logger,
final List<String> hosts,
final int limitPortCounts,
final TransportService transportService,
final TimeValue resolveTimeout) {
Objects.requireNonNull(executorService);
Expand All @@ -140,7 +138,7 @@ public static List<TransportAddress> resolveHostsLists(
final List<Callable<TransportAddress[]>> callables =
hosts
.stream()
.map(hn -> (Callable<TransportAddress[]>) () -> transportService.addressesFromString(hn, limitPortCounts))
.map(hn -> (Callable<TransportAddress[]>) () -> transportService.addressesFromString(hn))
.collect(Collectors.toList());
final List<Future<TransportAddress[]>> futures;
try {
Expand Down Expand Up @@ -224,9 +222,8 @@ protected void doRun() {
}

List<TransportAddress> providedAddresses
= hostsProvider.getSeedAddresses((hosts, limitPortCounts)
-> resolveHostsLists(executorService.get(), logger, hosts, limitPortCounts,
transportService, resolveTimeout));
= hostsProvider.getSeedAddresses(hosts ->
resolveHostsLists(executorService.get(), logger, hosts, transportService, resolveTimeout));

consumer.accept(providedAddresses);
}
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -50,12 +50,7 @@ public class SettingsBasedSeedHostsProvider implements SeedHostsProvider {
public static final Setting<List<String>> DISCOVERY_SEED_HOSTS_SETTING =
Setting.listSetting("discovery.seed_hosts", emptyList(), Function.identity(), Property.NodeScope);

// these limits are per-address
private static final int LIMIT_FOREIGN_PORTS_COUNT = 1;
private static final int LIMIT_LOCAL_PORTS_COUNT = 5;

private final List<String> configuredHosts;
private final int limitPortCounts;

public SettingsBasedSeedHostsProvider(Settings settings, TransportService transportService) {
if (LEGACY_DISCOVERY_ZEN_PING_UNICAST_HOSTS_SETTING.exists(settings)) {
Expand All @@ -66,22 +61,18 @@ public SettingsBasedSeedHostsProvider(Settings settings, TransportService transp
}
configuredHosts = LEGACY_DISCOVERY_ZEN_PING_UNICAST_HOSTS_SETTING.get(settings);
// we only limit to 1 address, makes no sense to ping 100 ports
limitPortCounts = LIMIT_FOREIGN_PORTS_COUNT;
} else if (DISCOVERY_SEED_HOSTS_SETTING.exists(settings)) {
configuredHosts = DISCOVERY_SEED_HOSTS_SETTING.get(settings);
// we only limit to 1 address, makes no sense to ping 100 ports
limitPortCounts = LIMIT_FOREIGN_PORTS_COUNT;
} else {
// if unicast hosts are not specified, fill with simple defaults on the local machine
configuredHosts = transportService.getLocalAddresses();
limitPortCounts = LIMIT_LOCAL_PORTS_COUNT;
configuredHosts = transportService.getDefaultSeedAddresses();
}

logger.debug("using initial hosts {}", configuredHosts);
}

@Override
public List<TransportAddress> getSeedAddresses(HostsResolver hostsResolver) {
return hostsResolver.resolveHosts(configuredHosts, limitPortCounts);
return hostsResolver.resolveHosts(configuredHosts);
}
}
Original file line number Diff line number Diff line change
Expand Up @@ -144,8 +144,7 @@ public UnicastZenPing(Settings settings, ThreadPool threadPool, TransportService
}

private SeedHostsProvider.HostsResolver createHostsResolver() {
return (hosts, limitPortCounts) -> SeedHostsResolver.resolveHostsLists(unicastZenPingExecutorService, logger, hosts,
limitPortCounts, transportService, resolveTimeout);
return hosts -> SeedHostsResolver.resolveHostsLists(unicastZenPingExecutorService, logger, hosts, transportService, resolveTimeout);
}

@Override
Expand Down
51 changes: 33 additions & 18 deletions server/src/main/java/org/elasticsearch/transport/TcpTransport.java
Original file line number Diff line number Diff line change
Expand Up @@ -86,6 +86,7 @@
import java.util.concurrent.locks.ReentrantReadWriteLock;
import java.util.regex.Matcher;
import java.util.regex.Pattern;
import java.util.stream.Collectors;

import static java.util.Collections.unmodifiableMap;
import static org.elasticsearch.common.transport.NetworkExceptionHelper.isCloseConnectionException;
Expand All @@ -102,6 +103,9 @@ public abstract class TcpTransport extends AbstractLifecycleComponent implements
private static final long NINETY_PER_HEAP_SIZE = (long) (JvmInfo.jvmInfo().getMem().getHeapMax().getBytes() * 0.9);
private static final BytesReference EMPTY_BYTES_REFERENCE = new BytesArray(new byte[0]);

// this limit is per-address
private static final int LIMIT_LOCAL_PORTS_COUNT = 6;

protected final Settings settings;
protected final ThreadPool threadPool;
protected final PageCacheRecycler pageCacheRecycler;
Expand Down Expand Up @@ -311,14 +315,20 @@ public Map<String, BoundTransportAddress> profileBoundAddresses() {
}

@Override
public List<String> getLocalAddresses() {
public List<String> getDefaultSeedAddresses() {
List<String> local = new ArrayList<>();
local.add("127.0.0.1");
// check if v6 is supported, if so, v4 will also work via mapped addresses.
if (NetworkUtils.SUPPORTS_V6) {
local.add("[::1]"); // may get ports appended!
}
return local;
return local.stream()
.flatMap(
address -> Arrays.stream(defaultPortRange())
.limit(LIMIT_LOCAL_PORTS_COUNT)
.mapToObj(port -> address + ":" + port)
)
.collect(Collectors.toList());
}

protected void bindServer(ProfileSettings profileSettings) {
Expand Down Expand Up @@ -456,8 +466,17 @@ static int resolvePublishPort(ProfileSettings profileSettings, List<InetSocketAd
}

@Override
public TransportAddress[] addressesFromString(String address, int perAddressLimit) throws UnknownHostException {
return parse(address, settings.get("transport.profiles.default.port", TransportSettings.PORT.get(settings)), perAddressLimit);
public TransportAddress[] addressesFromString(String address) throws UnknownHostException {
return parse(address, defaultPortRange()[0]);
}

private int[] defaultPortRange() {
return new PortsRange(
settings.get(
TransportSettings.PORT_PROFILE.getConcreteSettingForNamespace(TransportSettings.DEFAULT_PROFILE).getKey(),
TransportSettings.PORT.get(settings)
)
).ports();
}

// this code is a take on guava's HostAndPort, like a HostAndPortRange
Expand All @@ -467,9 +486,9 @@ public TransportAddress[] addressesFromString(String address, int perAddressLimi
private static final Pattern BRACKET_PATTERN = Pattern.compile("^\\[(.*:.*)\\](?::([\\d\\-]*))?$");

/**
* parse a hostname+port range spec into its equivalent addresses
* parse a hostname+port spec into its equivalent addresses
*/
static TransportAddress[] parse(String hostPortString, String defaultPortRange, int perAddressLimit) throws UnknownHostException {
static TransportAddress[] parse(String hostPortString, int defaultPort) throws UnknownHostException {
Objects.requireNonNull(hostPortString);
String host;
String portString = null;
Expand Down Expand Up @@ -498,22 +517,18 @@ static TransportAddress[] parse(String hostPortString, String defaultPortRange,
}
}

int port;
// if port isn't specified, fill with the default
if (portString == null || portString.isEmpty()) {
portString = defaultPortRange;
port = defaultPort;
} else {
port = Integer.parseInt(portString);
}

// generate address for each port in the range
Set<InetAddress> addresses = new HashSet<>(Arrays.asList(InetAddress.getAllByName(host)));
List<TransportAddress> transportAddresses = new ArrayList<>();
int[] ports = new PortsRange(portString).ports();
int limit = Math.min(ports.length, perAddressLimit);
for (int i = 0; i < limit; i++) {
for (InetAddress address : addresses) {
transportAddresses.add(new TransportAddress(address, ports[i]));
}
}
return transportAddresses.toArray(new TransportAddress[transportAddresses.size()]);
return Arrays.stream(InetAddress.getAllByName(host))
.distinct()
.map(address -> new TransportAddress(address, port))
.toArray(TransportAddress[]::new);
}

@Override
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -68,12 +68,12 @@ public interface Transport extends LifecycleComponent {
/**
* Returns an address from its string representation.
*/
TransportAddress[] addressesFromString(String address, int perAddressLimit) throws UnknownHostException;
TransportAddress[] addressesFromString(String address) throws UnknownHostException;

/**
* Returns a list of all local adresses for this transport
* Returns a list of all local addresses for this transport
*/
List<String> getLocalAddresses();
List<String> getDefaultSeedAddresses();

default CircuitBreaker getInFlightRequestBreaker() {
return new NoopCircuitBreaker("in-flight-noop");
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -313,8 +313,8 @@ public BoundTransportAddress boundAddress() {
return transport.boundAddress();
}

public List<String> getLocalAddresses() {
return transport.getLocalAddresses();
public List<String> getDefaultSeedAddresses() {
return transport.getDefaultSeedAddresses();
}

/**
Expand Down Expand Up @@ -750,8 +750,8 @@ private boolean shouldTraceAction(String action) {
return true;
}

public TransportAddress[] addressesFromString(String address, int perAddressLimit) throws UnknownHostException {
return transport.addressesFromString(address, perAddressLimit);
public TransportAddress[] addressesFromString(String address) throws UnknownHostException {
return transport.addressesFromString(address);
}

/**
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -170,7 +170,7 @@ public BoundTransportAddress boundAddress() {
}

@Override
public TransportAddress[] addressesFromString(String address, int perAddressLimit) throws UnknownHostException {
public TransportAddress[] addressesFromString(String address) throws UnknownHostException {
throw new UnsupportedOperationException();
}

Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -128,7 +128,7 @@ private static class TestIteration implements Closeable {
threadPool = new TestThreadPool("transport-client-nodes-service-tests");
transport = new FailAndRetryMockTransport<TestResponse>(random(), clusterName) {
@Override
public List<String> getLocalAddresses() {
public List<String> getDefaultSeedAddresses() {
return Collections.emptyList();
}

Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -401,7 +401,7 @@ public Map<String, BoundTransportAddress> profileBoundAddresses() {
}

@Override
public TransportAddress[] addressesFromString(String address, int perAddressLimit) {
public TransportAddress[] addressesFromString(String address) {
return new TransportAddress[0];
}

Expand Down Expand Up @@ -440,7 +440,7 @@ public boolean isClosed() {
}

@Override
public List<String> getLocalAddresses() {
public List<String> getDefaultSeedAddresses() {
return null;
}

Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -115,9 +115,8 @@ public void testEmptyUnicastHostsFile() throws Exception {

public void testUnicastHostsDoesNotExist() {
final FileBasedSeedHostsProvider provider = new FileBasedSeedHostsProvider(createTempDir().toAbsolutePath());
final List<TransportAddress> addresses = provider.getSeedAddresses((hosts, limitPortCounts) ->
SeedHostsResolver.resolveHostsLists(executorService, logger, hosts, limitPortCounts, transportService,
TimeValue.timeValueSeconds(10)));
final List<TransportAddress> addresses = provider.getSeedAddresses(hosts ->
SeedHostsResolver.resolveHostsLists(executorService, logger, hosts, transportService, TimeValue.timeValueSeconds(10)));
assertEquals(0, addresses.size());
}

Expand Down Expand Up @@ -145,8 +144,7 @@ private List<TransportAddress> setupAndRunHostProvider(final List<String> hostEn
writer.write(String.join("\n", hostEntries));
}

return new FileBasedSeedHostsProvider(configPath).getSeedAddresses((hosts, limitPortCounts) ->
SeedHostsResolver.resolveHostsLists(executorService, logger, hosts, limitPortCounts, transportService,
TimeValue.timeValueSeconds(10)));
return new FileBasedSeedHostsProvider(configPath).getSeedAddresses(hosts ->
SeedHostsResolver.resolveHostsLists(executorService, logger, hosts, transportService, TimeValue.timeValueSeconds(10)));
}
}
Loading