Skip to content
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

ZOOKEEPER-2184 Zookeeper Client should re-resolve hosts when connection attempts fail #534

Closed
wants to merge 7 commits into from
2 changes: 1 addition & 1 deletion build.xml
Original file line number Diff line number Diff line change
Expand Up @@ -39,7 +39,7 @@ xmlns:cs="antlib:com.puppycrawl.tools.checkstyle.ant">
<property name="netty.version" value="3.10.6.Final"/>

<property name="junit.version" value="4.12"/>
<property name="mockito.version" value="1.8.2"/>
<property name="mockito.version" value="1.8.5"/>
<property name="checkstyle.version" value="6.13"/>
<property name="commons-collections.version" value="3.2.2"/>

Expand Down
5 changes: 3 additions & 2 deletions src/java/main/org/apache/zookeeper/client/HostProvider.java
Original file line number Diff line number Diff line change
Expand Up @@ -34,8 +34,9 @@
*
* * The size() of a HostProvider may never be zero.
*
* A HostProvider must return resolved InetSocketAddress instances on next(),
* but it's up to the HostProvider, when it wants to do the resolving.
* A HostProvider must return resolved InetSocketAddress instances on next() if the next address is resolvable.
* In that case, it's up to the HostProvider, whether it returns the next resolvable address in the list or return
* the next one as UnResolved.
*
* Different HostProvider could be imagined:
*
Expand Down
121 changes: 77 additions & 44 deletions src/java/main/org/apache/zookeeper/client/StaticHostProvider.java
Original file line number Diff line number Diff line change
Expand Up @@ -22,6 +22,7 @@
import java.net.InetSocketAddress;
import java.net.UnknownHostException;
import java.util.ArrayList;
import java.util.Arrays;
import java.util.Collection;
import java.util.Collections;
import java.util.List;
Expand All @@ -32,11 +33,19 @@
import org.slf4j.LoggerFactory;

/**
* Most simple HostProvider, resolves only on instantiation.
*
* Most simple HostProvider, resolves on every next() call.
*
* Please be aware that although this class doesn't do any DNS caching, there're multiple levels of caching already
* present across the stack like in JVM, OS level, hardware, etc. The best we could do here is to get the most recent
* address from the underlying system which is considered up-to-date.
*
*/
@InterfaceAudience.Public
public final class StaticHostProvider implements HostProvider {
public interface Resolver {
InetAddress[] getAllByName(String name) throws UnknownHostException;
}

private static final Logger LOG = LoggerFactory
.getLogger(StaticHostProvider.class);

Expand Down Expand Up @@ -64,6 +73,8 @@ public final class StaticHostProvider implements HostProvider {

private float pOld, pNew;

private Resolver resolver;

/**
* Constructs a SimpleHostSet.
*
Expand All @@ -73,15 +84,29 @@ public final class StaticHostProvider implements HostProvider {
* if serverAddresses is empty or resolves to an empty list
*/
public StaticHostProvider(Collection<InetSocketAddress> serverAddresses) {
sourceOfRandomness = new Random(System.currentTimeMillis() ^ this.hashCode());
init(serverAddresses,
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Nit: Call StaticHostProvider(Collection serverAddresses, Resolver resolver).

Please change the other two constructor as well.

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Sorry @lvfangmin , I might be missing the point here. Shall I change the signature to use non-generic Collection?

Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

I think what @lvfangmin meant is instead of duplicating the code inside three constructors, only keep one concrete parameterized constructor implementation and let the other two invoke that one (with different parameters.).

Copy link
Contributor

@lvfangmin lvfangmin Jun 9, 2018

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

What @hanm said, instead of call init, call this.

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Tried that. Cannot call the other constructor, because this is referenced.

Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Yeah, I think referencing this before the object is constructed is prohibited. I think we can probably get around it but it looks like not worth the extra effort (I don't know a simple of way of doing it without dropping using the this.hashCode).
So I am ok with the current form of having all ctors invoking init method.

System.currentTimeMillis() ^ this.hashCode(),
new Resolver() {
@Override
public InetAddress[] getAllByName(String name) throws UnknownHostException {
return InetAddress.getAllByName(name);
}
});
}

this.serverAddresses = resolveAndShuffle(serverAddresses);
if (this.serverAddresses.isEmpty()) {
throw new IllegalArgumentException(
"A HostProvider may not be empty!");
}
currentIndex = -1;
lastIndex = -1;
/**
* Constructs a SimpleHostSet.
*
* Introduced for testing purposes. getAllByName() is a static method of InetAddress, therefore cannot be easily mocked.
* By abstraction of Resolver interface we can easily inject a mocked implementation in tests.
*
* @param serverAddresses
* possibly unresolved ZooKeeper server addresses
* @param resolver
* custom resolver implementation
*/
public StaticHostProvider(Collection<InetSocketAddress> serverAddresses, Resolver resolver) {
init(serverAddresses, System.currentTimeMillis() ^ this.hashCode(), resolver);
}

/**
Expand All @@ -96,36 +121,47 @@ public StaticHostProvider(Collection<InetSocketAddress> serverAddresses) {
*/
public StaticHostProvider(Collection<InetSocketAddress> serverAddresses,
long randomnessSeed) {
sourceOfRandomness = new Random(randomnessSeed);
init(serverAddresses, randomnessSeed, new Resolver() {
@Override
public InetAddress[] getAllByName(String name) throws UnknownHostException {
return InetAddress.getAllByName(name);
}
});
}

this.serverAddresses = resolveAndShuffle(serverAddresses);
if (this.serverAddresses.isEmpty()) {
private void init(Collection<InetSocketAddress> serverAddresses, long randomnessSeed, Resolver resolver) {
this.sourceOfRandomness = new Random(randomnessSeed);
this.resolver = resolver;
if (serverAddresses.isEmpty()) {
throw new IllegalArgumentException(
"A HostProvider may not be empty!");
}
}
this.serverAddresses = shuffle(serverAddresses);
currentIndex = -1;
lastIndex = -1;
lastIndex = -1;
}

private List<InetSocketAddress> resolveAndShuffle(Collection<InetSocketAddress> serverAddresses) {
List<InetSocketAddress> tmpList = new ArrayList<InetSocketAddress>(serverAddresses.size());
for (InetSocketAddress address : serverAddresses) {
try {
InetAddress ia = address.getAddress();
String addr = (ia != null) ? ia.getHostAddress() : address.getHostString();
InetAddress resolvedAddresses[] = InetAddress.getAllByName(addr);
for (InetAddress resolvedAddress : resolvedAddresses) {
InetAddress taddr = InetAddress.getByAddress(address.getHostString(), resolvedAddress.getAddress());
tmpList.add(new InetSocketAddress(taddr, address.getPort()));
}
} catch (UnknownHostException ex) {
LOG.warn("No IP address found for server: {}", address, ex);
private InetSocketAddress resolve(InetSocketAddress address) {
try {
String curHostString = address.getHostString();
List<InetAddress> resolvedAddresses = new ArrayList<>(Arrays.asList(this.resolver.getAllByName(curHostString)));
if (resolvedAddresses.isEmpty()) {
return address;
}
Collections.shuffle(resolvedAddresses);
return new InetSocketAddress(resolvedAddresses.get(0), address.getPort());
} catch (UnknownHostException e) {
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Should log the exception here. Does the caller handle the unknown address correctly?

Copy link
Contributor Author

@anmolnar anmolnar Jun 4, 2018

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

That's correct. The caller will end up getting UnknownHostException when trying to open the socket to the unresolvable address:

2018-06-04 12:31:26,022 [myid:huhuuhujkdshgfjksgd.com:2181] - WARN  [main-SendThread(huhuuhujkdshgfjksgd.com:2181):ClientCnxn$SendThread@1237] - Session 0x0 for server huhuuhujkdshgfjksgd.com:2181, unexpected error, closing socket connection and attempting reconnect
java.nio.channels.UnresolvedAddressException
	at sun.nio.ch.Net.checkAddress(Net.java:101)
	at sun.nio.ch.SocketChannelImpl.connect(SocketChannelImpl.java:622)
	at org.apache.zookeeper.ClientCnxnSocketNIO.registerAndConnect(ClientCnxnSocketNIO.java:275)
	at org.apache.zookeeper.ClientCnxnSocketNIO.connect(ClientCnxnSocketNIO.java:285)
	at org.apache.zookeeper.ClientCnxn$SendThread.startConnect(ClientCnxn.java:1091)
	at org.apache.zookeeper.ClientCnxn$SendThread.run(ClientCnxn.java:1133)

The reason for doing this way is to avoid API change of next().

Logging makes sense, I added an error log entry to make it clear.

LOG.error("Unable to resolve address: {}", address.toString(), e);
return address;
}
}

private List<InetSocketAddress> shuffle(Collection<InetSocketAddress> serverAddresses) {
List<InetSocketAddress> tmpList = new ArrayList<>(serverAddresses.size());
tmpList.addAll(serverAddresses);
Collections.shuffle(tmpList, sourceOfRandomness);
return tmpList;
}

}

/**
* Update the list of servers. This returns true if changing connections is necessary for load-balancing, false
Expand All @@ -149,15 +185,12 @@ private List<InetSocketAddress> resolveAndShuffle(Collection<InetSocketAddress>
* @param currentHost the host to which this client is currently connected
* @return true if changing connections is necessary for load-balancing, false otherwise
*/


@Override
public synchronized boolean updateServerList(
Collection<InetSocketAddress> serverAddresses,
InetSocketAddress currentHost) {
// Resolve server addresses and shuffle them
List<InetSocketAddress> resolvedList = resolveAndShuffle(serverAddresses);
if (resolvedList.isEmpty()) {
List<InetSocketAddress> shuffledList = shuffle(serverAddresses);
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

I am not sure why we change the resolveAndShuffle to shuffle here, they are semantically different (one tries to resolve address, the other does not and only does shuffle.). The serverAddresses passed in this method is unresolved address, and we need it resolved because we rely on the resolved addresses to compare the old / new server list (in the context of probability rebalancing clients for dynamic reconfiguration).

Without resolving I think the client rebalance logic will be broken. A side note that all tests still passed probably indicate we don't have a 100% coverage for the logic in our tests.

Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

After this change, the old / new server list both contains unresolved address, so it's comparable, but it's meaningless to compare addr.getAddress because it will return null when it's unresolved.

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

resolveAndShuffle has been split into two methods: resolve and shuffle, because we have to do it separately. Github shows the diff as original has been renamed and new one has been created.

I'll double check the rebalance functionality to make sure it's working properly and will also check related unit tests. I think what @lvfangmin is saying makes sense.

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Comparison works for unresolved addresses too, because of the last if condition here:

if (addr.getPort() == myServer.getPort()
    && ((addr.getAddress() != null
    && myServer.getAddress() != null && addr
    .getAddress().equals(myServer.getAddress())) || addr
    .getHostString().equals(myServer.getHostString()))) {

As long as getHostString() works for unresolved addresses, we're fine. However, exactly the same functionality has already been implemented in the InetSocketAddress.equals() method, so I refactored this part to use it.

Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

However, exactly the same functionality has already been implemented in the InetSocketAddress.equals() method

I am wondering if this is the case or not. I just did a random peek at https://github.com/openjdk-mirror/jdk7u-jdk/blob/master/src/share/classes/java/net/InetSocketAddress.java#L300, looks like if we compare an unresolved address with a resolved address, the equals will return false - but the code you pasted will return true if getHostString works for both resolved and unresolved address... could you double check this behavior?

Also, is it possible to add a test case to cover the case where the second parameter of updateServerList is a resolved address? The existing test cases only cover the case where the second parameter (myServer) is unresolved. In practice I think the method updateServerList is called by ZooKeeper's updateServerList method with second parameter as a resolved address (the remote server where current client connected to.).

Copy link
Contributor Author

@anmolnar anmolnar Jul 11, 2018

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

@hanm Good catch, that makes perfect sense. The implementation is slightly different.

I added 4 new test cases to cover the different combinations of the 2nd and 3rd parameter. Some of them could be redundant (as you said tests present already), but I've already opened a Jira to clean-up this test file, so it should be okay. Test scenarios:

Given: list of servers contains 1 element, client is connected (currentHost (myServer) != null), trying to replace server list with the same address (replaceHost).

New impl

  1. currentHost resolved, replaceHost unresolved => client should disconnect,
  2. currentHost resolved, replaceHost resolved => client should not disconnect,
  3. currentHost unresolved, replaceHost unresolved => client should disconnect,
  4. currentHost unresolved, replaceHost resolved => client should disconnect

Old impl

  1. currentHost resolved, replaceHost unresolved => client should not disconnect,
  2. currentHost resolved, replaceHost resolved => client should not disconnect,
  3. currentHost unresolved, replaceHost unresolved => client should disconnect,
  4. currentHost unresolved, replaceHost resolved => client should not disconnect

Basically the difference is in the case when comparing resolved address with unresolved. The built-in implementation treats them as different address, hence forcing the client to disconnect which makes slightly more sense to me.

Anyway I'm happy to revert the change if you think this is not acceptable.

Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Thanks for quick update, @anmolnar !

The built-in implementation treats them as different address

That's what I am not sure about. If an unresolved address and a resolved address actually maps same address (after unresolved address gets resolved), should both be treated as same address (old behavior, via getHostString), instead of different address?

What I am thinking is to instead of using equal - use the old verbose code which will work (so far) for all combinations of comparing resolved and unresolved address. What do you think about this?

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

My only concern is with the first case: IP address might have changed of the host, so client should disconnect and trigger a re-resolution.

Anyway. No issue has been reported with the logic we're talking about, so let's just leave it out from this patch. Reverted.

if (shuffledList.isEmpty()) {
throw new IllegalArgumentException(
"A HostProvider may not be empty!");
}
Expand All @@ -183,7 +216,7 @@ public synchronized boolean updateServerList(
}
}

for (InetSocketAddress addr : resolvedList) {
for (InetSocketAddress addr : shuffledList) {
if (addr.getPort() == myServer.getPort()
&& ((addr.getAddress() != null
&& myServer.getAddress() != null && addr
Expand All @@ -200,11 +233,11 @@ public synchronized boolean updateServerList(
oldServers.clear();
// Divide the new servers into oldServers that were in the previous list
// and newServers that were not in the previous list
for (InetSocketAddress resolvedAddress : resolvedList) {
if (this.serverAddresses.contains(resolvedAddress)) {
oldServers.add(resolvedAddress);
for (InetSocketAddress address : shuffledList) {
if (this.serverAddresses.contains(address)) {
oldServers.add(address);
} else {
newServers.add(resolvedAddress);
newServers.add(address);
}
}

Expand Down Expand Up @@ -245,11 +278,11 @@ public synchronized boolean updateServerList(
}

if (!reconfigMode) {
currentIndex = resolvedList.indexOf(getServerAtCurrentIndex());
currentIndex = shuffledList.indexOf(getServerAtCurrentIndex());
} else {
currentIndex = -1;
}
this.serverAddresses = resolvedList;
this.serverAddresses = shuffledList;
currentIndexOld = -1;
currentIndexNew = -1;
lastIndex = currentIndex;
Expand Down Expand Up @@ -314,7 +347,7 @@ public InetSocketAddress next(long spinDelay) {
addr = nextHostInReconfigMode();
if (addr != null) {
currentIndex = serverAddresses.indexOf(addr);
return addr;
return resolve(addr);
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Can you do some test to find out performance/latency of resolving an addr? If it's costly, maybe we should cache the resolved one instead of resolve it every time.

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

I disagree. We should not do any caching in our codebase, because there're multiple levels of caching already present in DNS infrastructure, like JVM caching, os-level caching, DNS servers caching, etc. resolve() will eventually become a no-op if any of these caches find a hit.

Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

I know there is DNS caching, not aware that we have JVM or os caching for this, if we tested this is trivial, I totally agree we should keep this simple.

Copy link
Contributor Author

@anmolnar anmolnar Jun 18, 2018

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

IMHO JVM caching is overkill too. It's not application's responsibility to deal with DNS re-resolution, because it shouldn't happen in failure scenarios only. On the flipside to avoid DNS server flooding some caching in the app would be reasonable and JVM is a standard way of doing that.

}
//tried all servers and couldn't connect
reconfigMode = false;
Expand All @@ -339,7 +372,7 @@ public InetSocketAddress next(long spinDelay) {
}
}

return addr;
return resolve(addr);
}

public synchronized void onConnected() {
Expand Down
Loading