You are viewing a plain text version of this content. The canonical link for it is here.
Posted to dev@zookeeper.apache.org by anmolnar <gi...@git.apache.org> on 2018/01/24 14:16:50 UTC

[GitHub] zookeeper pull request #451: ZOOKEEPER-2184: Zookeeper Client should re-reso...

GitHub user anmolnar opened a pull request:

    https://github.com/apache/zookeeper/pull/451

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

    This one is the pick-up of @fpj 's original PR: #150 
    Targeting and rebased on the 3.4 branch.

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

    $ git pull https://github.com/anmolnar/zookeeper ZOOKEEPER-2184

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

    https://github.com/apache/zookeeper/pull/451.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 #451
    
----
commit 229760d7757f47e271a8e059c1aeac10f0847a2a
Author: fpj <fp...@...>
Date:   2017-01-14T16:58:15Z

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

----


---

[GitHub] zookeeper pull request #451: ZOOKEEPER-2184: Zookeeper Client should re-reso...

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

    https://github.com/apache/zookeeper/pull/451#discussion_r167027624
  
    --- Diff: src/java/test/org/apache/zookeeper/test/StaticHostProviderTest.java ---
    @@ -117,8 +116,32 @@ public void testTwoInvalidHostAddresses() {
             list.add(new InetSocketAddress("a", 2181));
             list.add(new InetSocketAddress("b", 2181));
             new StaticHostProvider(list);
    +	}
    +
    +    @Test
    +    public void testReResolving() {
    +        byte size = 1;
    +        ArrayList<InetSocketAddress> list = new ArrayList<InetSocketAddress>(size);
    +
    +        // Test a hostname that resolves to multiple addresses
    +        list.add(InetSocketAddress.createUnresolved("www.apache.org", 1234));
    --- End diff --
    
    Makes sense, I'm working on it.


---

[GitHub] zookeeper issue #451: ZOOKEEPER-2184: Zookeeper Client should re-resolve hos...

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

    https://github.com/apache/zookeeper/pull/451
  
    Also, as I'm sure many are wondering, a discussion about the next release including this fix is taking place in the mailing list:
    
    https://lists.apache.org/thread.html/8d3dcd84b0ad47563e2cf2619eb0775066ff975453865938dd4e7380@%3Cdev.zookeeper.apache.org%3E


---

[GitHub] zookeeper pull request #451: ZOOKEEPER-2184: Zookeeper Client should re-reso...

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

    https://github.com/apache/zookeeper/pull/451#discussion_r166102194
  
    --- Diff: src/java/main/org/apache/zookeeper/client/StaticHostProvider.java ---
    @@ -58,48 +61,122 @@
         public StaticHostProvider(Collection<InetSocketAddress> serverAddresses) {
             for (InetSocketAddress address : serverAddresses) {
                 try {
    -                InetAddress ia = address.getAddress();
    -                InetAddress resolvedAddresses[] = InetAddress.getAllByName((ia != null) ? ia.getHostAddress() :
    -                        address.getHostName());
    +                InetAddress resolvedAddresses[] = InetAddress.getAllByName(getHostString(address));
                     for (InetAddress resolvedAddress : resolvedAddresses) {
    -                    // If hostName is null but the address is not, we can tell that
    -                    // the hostName is an literal IP address. Then we can set the host string as the hostname
    -                    // safely to avoid reverse DNS lookup.
    -                    // As far as i know, the only way to check if the hostName is null is use toString().
    -                    // Both the two implementations of InetAddress are final class, so we can trust the return value of
    -                    // the toString() method.
    -                    if (resolvedAddress.toString().startsWith("/")
    -                            && resolvedAddress.getAddress() != null) {
    -                        this.serverAddresses.add(
    -                                new InetSocketAddress(InetAddress.getByAddress(
    -                                        address.getHostName(),
    -                                        resolvedAddress.getAddress()),
    -                                        address.getPort()));
    -                    } else {
    -                        this.serverAddresses.add(new InetSocketAddress(resolvedAddress.getHostAddress(), address.getPort()));
    -                    }
    +                    this.serverAddresses.add(new InetSocketAddress(resolvedAddress, address.getPort()));
                     }
                 } catch (UnknownHostException e) {
                     LOG.error("Unable to connect to server: {}", address, e);
                 }
             }
    -        
    +
             if (this.serverAddresses.isEmpty()) {
                 throw new IllegalArgumentException(
                         "A HostProvider may not be empty!");
             }
             Collections.shuffle(this.serverAddresses);
         }
     
    +    /**
    +     * Evaluate to a hostname if one is available and otherwise it returns the
    +     * string representation of the IP address.
    +     *
    +     * In Java 7, we have a method getHostString, but earlier versions do not support it.
    +     * This method is to provide a replacement for InetSocketAddress.getHostString().
    +     *
    +     * @param addr
    +     * @return Hostname string of address parameter
    +     */
    +    private String getHostString(InetSocketAddress addr) {
    +        String hostString = "";
    +
    +        if (addr == null) {
    +            return hostString;
    +        }
    +        if (!addr.isUnresolved()) {
    +            InetAddress ia = addr.getAddress();
    +
    +            // If the string starts with '/', then it has no hostname
    +            // and we want to avoid the reverse lookup, so we return
    +            // the string representation of the address.
    +            if (ia.toString().startsWith("/")) {
    +                hostString = ia.getHostAddress();
    +            } else {
    +                hostString = addr.getHostName();
    +            }
    +        } else {
    +            // According to the Java 6 documentation, if the hostname is
    +            // unresolved, then the string before the colon is the hostname.
    +            String addrString = addr.toString();
    +            hostString = addrString.substring(0, addrString.lastIndexOf(':'));
    +        }
    +
    +        return hostString;
    +    }
    +
         public int size() {
             return serverAddresses.size();
         }
     
    +    // Counts the number of addresses added and removed during
    +    // the last call to next. Used mainly for test purposes.
    +    // See StasticHostProviderTest.
    +    private int nextAdded = 0;
    +    private int nextRemoved = 0;
    +
    +    public int getNextAdded() {
    +        return nextAdded;
    +    }
    +
    +    public int getNextRemoved() {
    +        return nextRemoved;
    +    }
    +
         public InetSocketAddress next(long spinDelay) {
    -        ++currentIndex;
    -        if (currentIndex == serverAddresses.size()) {
    -            currentIndex = 0;
    +        // Handle possible connection error by re-resolving hostname if possible
    +        if (!connectedSinceNext) {
    +            InetSocketAddress curAddr = serverAddresses.get(currentIndex);
    +            String curHostString = getHostString(curAddr);
    +            if (!curHostString.equals(curAddr.getAddress().getHostAddress())) {
    +                LOG.info("Resolving again hostname: {}", getHostString(curAddr));
    +                try {
    +                    int thePort = curAddr.getPort();
    +                    InetAddress resolvedAddresses[] = InetAddress.getAllByName(curHostString);
    +                    nextAdded = 0;
    +                    nextRemoved = 0;
    +                    if (resolvedAddresses.length == 1) {
    --- End diff --
    
    what happens when a host that resolves to multiple addresses changes to resolving to just one?


---

[GitHub] zookeeper pull request #451: ZOOKEEPER-2184: Zookeeper Client should re-reso...

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

    https://github.com/apache/zookeeper/pull/451#discussion_r166097161
  
    --- Diff: src/java/main/org/apache/zookeeper/client/StaticHostProvider.java ---
    @@ -6,9 +6,9 @@
      * to you under the Apache License, Version 2.0 (the
      * "License"); you may not use this file except in compliance
      * with the License.  You may obtain a copy of the License at
    - *
    - *     http://www.apache.org/licenses/LICENSE-2.0
    - *
    + * <p>
    --- End diff --
    
    was this accidental?


---

[GitHub] zookeeper pull request #451: ZOOKEEPER-2184: Zookeeper Client should re-reso...

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

    https://github.com/apache/zookeeper/pull/451#discussion_r166476703
  
    --- Diff: src/java/main/org/apache/zookeeper/client/StaticHostProvider.java ---
    @@ -58,48 +61,122 @@
         public StaticHostProvider(Collection<InetSocketAddress> serverAddresses) {
             for (InetSocketAddress address : serverAddresses) {
                 try {
    -                InetAddress ia = address.getAddress();
    -                InetAddress resolvedAddresses[] = InetAddress.getAllByName((ia != null) ? ia.getHostAddress() :
    -                        address.getHostName());
    +                InetAddress resolvedAddresses[] = InetAddress.getAllByName(getHostString(address));
                     for (InetAddress resolvedAddress : resolvedAddresses) {
    -                    // If hostName is null but the address is not, we can tell that
    -                    // the hostName is an literal IP address. Then we can set the host string as the hostname
    -                    // safely to avoid reverse DNS lookup.
    -                    // As far as i know, the only way to check if the hostName is null is use toString().
    -                    // Both the two implementations of InetAddress are final class, so we can trust the return value of
    -                    // the toString() method.
    -                    if (resolvedAddress.toString().startsWith("/")
    -                            && resolvedAddress.getAddress() != null) {
    -                        this.serverAddresses.add(
    -                                new InetSocketAddress(InetAddress.getByAddress(
    -                                        address.getHostName(),
    -                                        resolvedAddress.getAddress()),
    -                                        address.getPort()));
    -                    } else {
    -                        this.serverAddresses.add(new InetSocketAddress(resolvedAddress.getHostAddress(), address.getPort()));
    -                    }
    +                    this.serverAddresses.add(new InetSocketAddress(resolvedAddress, address.getPort()));
                     }
                 } catch (UnknownHostException e) {
                     LOG.error("Unable to connect to server: {}", address, e);
                 }
             }
    -        
    +
             if (this.serverAddresses.isEmpty()) {
                 throw new IllegalArgumentException(
                         "A HostProvider may not be empty!");
             }
             Collections.shuffle(this.serverAddresses);
         }
     
    +    /**
    +     * Evaluate to a hostname if one is available and otherwise it returns the
    +     * string representation of the IP address.
    +     *
    +     * In Java 7, we have a method getHostString, but earlier versions do not support it.
    +     * This method is to provide a replacement for InetSocketAddress.getHostString().
    +     *
    +     * @param addr
    +     * @return Hostname string of address parameter
    +     */
    +    private String getHostString(InetSocketAddress addr) {
    +        String hostString = "";
    +
    +        if (addr == null) {
    +            return hostString;
    +        }
    +        if (!addr.isUnresolved()) {
    +            InetAddress ia = addr.getAddress();
    +
    +            // If the string starts with '/', then it has no hostname
    +            // and we want to avoid the reverse lookup, so we return
    +            // the string representation of the address.
    +            if (ia.toString().startsWith("/")) {
    +                hostString = ia.getHostAddress();
    +            } else {
    +                hostString = addr.getHostName();
    +            }
    +        } else {
    +            // According to the Java 6 documentation, if the hostname is
    +            // unresolved, then the string before the colon is the hostname.
    +            String addrString = addr.toString();
    +            hostString = addrString.substring(0, addrString.lastIndexOf(':'));
    +        }
    +
    +        return hostString;
    +    }
    +
         public int size() {
             return serverAddresses.size();
         }
     
    +    // Counts the number of addresses added and removed during
    +    // the last call to next. Used mainly for test purposes.
    +    // See StasticHostProviderTest.
    +    private int nextAdded = 0;
    +    private int nextRemoved = 0;
    +
    +    public int getNextAdded() {
    +        return nextAdded;
    +    }
    +
    +    public int getNextRemoved() {
    +        return nextRemoved;
    +    }
    +
         public InetSocketAddress next(long spinDelay) {
    -        ++currentIndex;
    -        if (currentIndex == serverAddresses.size()) {
    -            currentIndex = 0;
    +        // Handle possible connection error by re-resolving hostname if possible
    +        if (!connectedSinceNext) {
    +            InetSocketAddress curAddr = serverAddresses.get(currentIndex);
    +            String curHostString = getHostString(curAddr);
    +            if (!curHostString.equals(curAddr.getAddress().getHostAddress())) {
    +                LOG.info("Resolving again hostname: {}", getHostString(curAddr));
    +                try {
    +                    int thePort = curAddr.getPort();
    +                    InetAddress resolvedAddresses[] = InetAddress.getAllByName(curHostString);
    +                    nextAdded = 0;
    +                    nextRemoved = 0;
    +                    if (resolvedAddresses.length == 1) {
    +                        serverAddresses.set(currentIndex, new InetSocketAddress(resolvedAddresses[0], thePort));
    +                        nextAdded = nextRemoved = 1;
    +                        LOG.debug("Newly resolved address: {}", resolvedAddresses[0]);
    +                    } else {
    +                        int i = 0;
    +                        while (i < serverAddresses.size()) {
    +                            if (getHostString(serverAddresses.get(i)).equals(curHostString) &&
    --- End diff --
    
    Not sure I understand this point. I believe there are implementations of HashMap that provide predictable ordering of the keys, so you should still be able to use next. Let me know if I am misunderstanding you here. 


---

[GitHub] zookeeper pull request #451: ZOOKEEPER-2184: Zookeeper Client should re-reso...

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

    https://github.com/apache/zookeeper/pull/451#discussion_r168575513
  
    --- Diff: src/java/test/org/apache/zookeeper/test/ReadOnlyModeTest.java ---
    @@ -239,13 +243,13 @@ public void testSessionEstablishment() throws Exception {
         public void testSeekForRwServer() throws Exception {
     
             // setup the logger to capture all logs
    -        Layout layout = Logger.getRootLogger().getAppender("CONSOLE")
    +        Layout layout = org.apache.log4j.Logger.getRootLogger().getAppender("CONSOLE")
    --- End diff --
    
    I guess my real question is, do we need any of the changes in this file?


---

[GitHub] zookeeper issue #451: ZOOKEEPER-2184: Zookeeper Client should re-resolve hos...

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

    https://github.com/apache/zookeeper/pull/451
  
    Committed to branch-3.4. Please close the pull request @anmolnar.
    
    Please port it to branch-3.5 and master (if you like :-), and create necessary JIRAs for auxiliary tasks (documentation update, etc). 


---

[GitHub] zookeeper issue #451: ZOOKEEPER-2184: Zookeeper Client should re-resolve hos...

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

    https://github.com/apache/zookeeper/pull/451
  
    @phunt @afine Did you have a chance to take a look?
    I think we've addressed all issues that were mentioned in the original PR.


---

[GitHub] zookeeper pull request #451: ZOOKEEPER-2184: Zookeeper Client should re-reso...

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

    https://github.com/apache/zookeeper/pull/451#discussion_r165658965
  
    --- Diff: src/java/main/org/apache/zookeeper/client/StaticHostProvider.java ---
    @@ -57,29 +62,12 @@
          */
         public StaticHostProvider(Collection<InetSocketAddress> serverAddresses) {
             for (InetSocketAddress address : serverAddresses) {
    -            try {
    -                InetAddress ia = address.getAddress();
    -                InetAddress resolvedAddresses[] = InetAddress.getAllByName((ia != null) ? ia.getHostAddress() :
    -                        address.getHostName());
    -                for (InetAddress resolvedAddress : resolvedAddresses) {
    -                    // If hostName is null but the address is not, we can tell that
    -                    // the hostName is an literal IP address. Then we can set the host string as the hostname
    -                    // safely to avoid reverse DNS lookup.
    -                    // As far as i know, the only way to check if the hostName is null is use toString().
    -                    // Both the two implementations of InetAddress are final class, so we can trust the return value of
    -                    // the toString() method.
    -                    if (resolvedAddress.toString().startsWith("/")
    -                            && resolvedAddress.getAddress() != null) {
    -                        this.serverAddresses.add(
    -                                new InetSocketAddress(InetAddress.getByAddress(
    -                                        address.getHostName(),
    -                                        resolvedAddress.getAddress()),
    -                                        address.getPort()));
    -                    } else {
    -                        this.serverAddresses.add(new InetSocketAddress(resolvedAddress.getHostAddress(), address.getPort()));
    -                    }
    -                }
    -            } catch (UnknownHostException e) {
    +			try {
    --- End diff --
    
    Fixing.


---

[GitHub] zookeeper pull request #451: ZOOKEEPER-2184: Zookeeper Client should re-reso...

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

    https://github.com/apache/zookeeper/pull/451#discussion_r186591847
  
    --- Diff: src/java/main/org/apache/zookeeper/client/StaticHostProvider.java ---
    @@ -111,9 +154,18 @@ public InetSocketAddress next(long spinDelay) {
                 lastIndex = 0;
             }
     
    -        return serverAddresses.get(currentIndex);
    +        InetSocketAddress curAddr = serverAddresses.get(currentIndex);
    +
    +        String curHostString = getHostString(curAddr);
    +        List<InetAddress> resolvedAddresses = new ArrayList<InetAddress>(Arrays.asList(this.resolver.getAllByName(curHostString)));
    +        if (resolvedAddresses.isEmpty()) {
    +            throw new UnknownHostException("No IP address returned for address: " + curHostString);
    +        }
    +        Collections.shuffle(resolvedAddresses);
    --- End diff --
    
    Correctly configured DNS servers return multiple IP addresses in round robin fashion for a single hostname, so shuffle is not needed. However there was a concern of badly configured DNS servers might not do this properly and we want to workaround it here.


---

[GitHub] zookeeper pull request #451: ZOOKEEPER-2184: Zookeeper Client should re-reso...

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

    https://github.com/apache/zookeeper/pull/451#discussion_r165527366
  
    --- Diff: src/java/main/org/apache/zookeeper/client/StaticHostProvider.java ---
    @@ -57,29 +62,12 @@
          */
         public StaticHostProvider(Collection<InetSocketAddress> serverAddresses) {
             for (InetSocketAddress address : serverAddresses) {
    -            try {
    -                InetAddress ia = address.getAddress();
    -                InetAddress resolvedAddresses[] = InetAddress.getAllByName((ia != null) ? ia.getHostAddress() :
    -                        address.getHostName());
    -                for (InetAddress resolvedAddress : resolvedAddresses) {
    -                    // If hostName is null but the address is not, we can tell that
    -                    // the hostName is an literal IP address. Then we can set the host string as the hostname
    -                    // safely to avoid reverse DNS lookup.
    -                    // As far as i know, the only way to check if the hostName is null is use toString().
    -                    // Both the two implementations of InetAddress are final class, so we can trust the return value of
    -                    // the toString() method.
    -                    if (resolvedAddress.toString().startsWith("/")
    -                            && resolvedAddress.getAddress() != null) {
    -                        this.serverAddresses.add(
    -                                new InetSocketAddress(InetAddress.getByAddress(
    -                                        address.getHostName(),
    -                                        resolvedAddress.getAddress()),
    -                                        address.getPort()));
    -                    } else {
    -                        this.serverAddresses.add(new InetSocketAddress(resolvedAddress.getHostAddress(), address.getPort()));
    -                    }
    -                }
    -            } catch (UnknownHostException e) {
    +			try {
    --- End diff --
    
    something is wrong with the indentation here


---

[GitHub] zookeeper issue #451: ZOOKEEPER-2184: Zookeeper Client should re-resolve hos...

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

    https://github.com/apache/zookeeper/pull/451
  
    I am reviewing this and will merge before Memorial Day if no other issues.


---

[GitHub] zookeeper pull request #451: ZOOKEEPER-2184: Zookeeper Client should re-reso...

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

    https://github.com/apache/zookeeper/pull/451#discussion_r165666521
  
    --- Diff: src/java/test/org/apache/zookeeper/client/StaticHostProviderTest.java ---
    @@ -16,7 +16,7 @@
      * limitations under the License.
      */
     
    -package org.apache.zookeeper.test;
    +package org.apache.zookeeper.client;
    --- End diff --
    
    It was move to client package, because it uses package-private methods of StaticHostProvider. I moved back to 'test' package and change the affected methods to public.


---

[GitHub] zookeeper pull request #451: ZOOKEEPER-2184: Zookeeper Client should re-reso...

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

    https://github.com/apache/zookeeper/pull/451#discussion_r168580127
  
    --- Diff: src/java/main/org/apache/zookeeper/client/StaticHostProvider.java ---
    @@ -58,48 +61,122 @@
         public StaticHostProvider(Collection<InetSocketAddress> serverAddresses) {
             for (InetSocketAddress address : serverAddresses) {
                 try {
    -                InetAddress ia = address.getAddress();
    -                InetAddress resolvedAddresses[] = InetAddress.getAllByName((ia != null) ? ia.getHostAddress() :
    -                        address.getHostName());
    +                InetAddress resolvedAddresses[] = InetAddress.getAllByName(getHostString(address));
                     for (InetAddress resolvedAddress : resolvedAddresses) {
    -                    // If hostName is null but the address is not, we can tell that
    -                    // the hostName is an literal IP address. Then we can set the host string as the hostname
    -                    // safely to avoid reverse DNS lookup.
    -                    // As far as i know, the only way to check if the hostName is null is use toString().
    -                    // Both the two implementations of InetAddress are final class, so we can trust the return value of
    -                    // the toString() method.
    -                    if (resolvedAddress.toString().startsWith("/")
    -                            && resolvedAddress.getAddress() != null) {
    -                        this.serverAddresses.add(
    -                                new InetSocketAddress(InetAddress.getByAddress(
    -                                        address.getHostName(),
    -                                        resolvedAddress.getAddress()),
    -                                        address.getPort()));
    -                    } else {
    -                        this.serverAddresses.add(new InetSocketAddress(resolvedAddress.getHostAddress(), address.getPort()));
    -                    }
    +                    this.serverAddresses.add(new InetSocketAddress(resolvedAddress, address.getPort()));
                     }
                 } catch (UnknownHostException e) {
                     LOG.error("Unable to connect to server: {}", address, e);
                 }
             }
    -        
    +
             if (this.serverAddresses.isEmpty()) {
                 throw new IllegalArgumentException(
                         "A HostProvider may not be empty!");
             }
             Collections.shuffle(this.serverAddresses);
         }
     
    +    /**
    +     * Evaluate to a hostname if one is available and otherwise it returns the
    +     * string representation of the IP address.
    +     *
    +     * In Java 7, we have a method getHostString, but earlier versions do not support it.
    +     * This method is to provide a replacement for InetSocketAddress.getHostString().
    +     *
    +     * @param addr
    +     * @return Hostname string of address parameter
    +     */
    +    private String getHostString(InetSocketAddress addr) {
    +        String hostString = "";
    +
    +        if (addr == null) {
    +            return hostString;
    +        }
    +        if (!addr.isUnresolved()) {
    +            InetAddress ia = addr.getAddress();
    +
    +            // If the string starts with '/', then it has no hostname
    +            // and we want to avoid the reverse lookup, so we return
    +            // the string representation of the address.
    +            if (ia.toString().startsWith("/")) {
    +                hostString = ia.getHostAddress();
    +            } else {
    +                hostString = addr.getHostName();
    +            }
    +        } else {
    +            // According to the Java 6 documentation, if the hostname is
    +            // unresolved, then the string before the colon is the hostname.
    +            String addrString = addr.toString();
    +            hostString = addrString.substring(0, addrString.lastIndexOf(':'));
    +        }
    +
    +        return hostString;
    +    }
    +
         public int size() {
             return serverAddresses.size();
         }
     
    +    // Counts the number of addresses added and removed during
    +    // the last call to next. Used mainly for test purposes.
    +    // See StasticHostProviderTest.
    +    private int nextAdded = 0;
    +    private int nextRemoved = 0;
    +
    +    public int getNextAdded() {
    +        return nextAdded;
    +    }
    +
    +    public int getNextRemoved() {
    +        return nextRemoved;
    +    }
    +
         public InetSocketAddress next(long spinDelay) {
    -        ++currentIndex;
    -        if (currentIndex == serverAddresses.size()) {
    -            currentIndex = 0;
    +        // Handle possible connection error by re-resolving hostname if possible
    +        if (!connectedSinceNext) {
    +            InetSocketAddress curAddr = serverAddresses.get(currentIndex);
    +            String curHostString = getHostString(curAddr);
    +            if (!curHostString.equals(curAddr.getAddress().getHostAddress())) {
    +                LOG.info("Resolving again hostname: {}", getHostString(curAddr));
    +                try {
    +                    int thePort = curAddr.getPort();
    +                    InetAddress resolvedAddresses[] = InetAddress.getAllByName(curHostString);
    +                    nextAdded = 0;
    +                    nextRemoved = 0;
    +                    if (resolvedAddresses.length == 1) {
    +                        serverAddresses.set(currentIndex, new InetSocketAddress(resolvedAddresses[0], thePort));
    +                        nextAdded = nextRemoved = 1;
    +                        LOG.debug("Newly resolved address: {}", resolvedAddresses[0]);
    +                    } else {
    +                        int i = 0;
    +                        while (i < serverAddresses.size()) {
    +                            if (getHostString(serverAddresses.get(i)).equals(curHostString) &&
    --- End diff --
    
    That's exactly the point. 
    (think I've already suggested the same thing in an outdated post)


---

[GitHub] zookeeper pull request #451: ZOOKEEPER-2184: Zookeeper Client should re-reso...

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

    https://github.com/apache/zookeeper/pull/451#discussion_r165529085
  
    --- Diff: src/java/main/org/apache/zookeeper/client/StaticHostProvider.java ---
    @@ -91,15 +79,106 @@ public StaticHostProvider(Collection<InetSocketAddress> serverAddresses) {
             Collections.shuffle(this.serverAddresses);
         }
     
    +    /**
    +     * Evaluate to a hostname if one is available and otherwise it returns the
    +     * string representation of the IP address.
    +     *
    +     * In Java 7, we have a method getHostString, but earlier versions do not support it.
    +     * This method is to provide a replacement for InetSocketAddress.getHostString().
    +     *
    +     * @param addr
    +     * @return Hostname string of address parameter
    +     */
    +    private String getHostString(InetSocketAddress addr) {
    +        String hostString = "";
    +
    +        if (addr == null) {
    +            return hostString;
    +        }
    +        if (!addr.isUnresolved()) {
    +            InetAddress ia = addr.getAddress();
    +
    +            // If the string starts with '/', then it has no hostname
    +            // and we want to avoid the reverse lookup, so we return
    +            // the string representation of the address.
    +            if (ia.toString().startsWith("/")) {
    +                hostString = ia.getHostAddress();
    +            } else {
    +                hostString = addr.getHostName();
    +            }
    +        } else {
    +            // According to the Java 6 documentation, if the hostname is
    +            // unresolved, then the string before the colon is the hostname.
    +            String addrString = addr.toString();
    +            hostString = addrString.substring(0, addrString.lastIndexOf(':'));
    +        }
    +
    +        return hostString;
    +    }
    +
         public int size() {
             return serverAddresses.size();
         }
     
    +    // Counts the number of addresses added and removed during
    +    // the last call to next. Used mainly for test purposes.
    +    // See StasticHostProviderTest.
    +    private int nextAdded = 0;
    +    private int nextRemoved = 0;
    +
    +    int getNextAdded() {
    +        return nextAdded;
    +    }
    +
    +    int getNextRemoved() {
    +        return nextRemoved;
    +    }
    +
         public InetSocketAddress next(long spinDelay) {
    -        ++currentIndex;
    -        if (currentIndex == serverAddresses.size()) {
    -            currentIndex = 0;
    +        // Handle possible connection error by re-resolving hostname if possible
    +        if (!connectedSinceNext) {
    --- End diff --
    
    would you mind explaining exactly under which conditions we reresolve the hostname and under which conditions we try the next one in the host list? My reading is that this reresolves everything if the client fails to connect to two hosts in a row. Is this the desired behavior?
    
    And do we always reresolve all serverAddresses?


---

[GitHub] zookeeper pull request #451: ZOOKEEPER-2184: Zookeeper Client should re-reso...

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

    https://github.com/apache/zookeeper/pull/451#discussion_r186553004
  
    --- Diff: src/java/main/org/apache/zookeeper/client/StaticHostProvider.java ---
    @@ -111,9 +154,18 @@ public InetSocketAddress next(long spinDelay) {
                 lastIndex = 0;
             }
     
    -        return serverAddresses.get(currentIndex);
    +        InetSocketAddress curAddr = serverAddresses.get(currentIndex);
    +
    +        String curHostString = getHostString(curAddr);
    +        List<InetAddress> resolvedAddresses = new ArrayList<InetAddress>(Arrays.asList(this.resolver.getAllByName(curHostString)));
    +        if (resolvedAddresses.isEmpty()) {
    +            throw new UnknownHostException("No IP address returned for address: " + curHostString);
    --- End diff --
    
    Should we try with the next on the list rather than throwing? After all, the call is to get the "next" resolved address. Of course, it is possible that we go through the whole list an find nothing. We might want to throw then so that we don't loop forever.


---

[GitHub] zookeeper issue #451: ZOOKEEPER-2184: Zookeeper Client should re-resolve hos...

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

    https://github.com/apache/zookeeper/pull/451
  
    The patch looks good, I think we just need to get the public API change issue settled before we can merge this in.


---

[GitHub] zookeeper pull request #451: ZOOKEEPER-2184: Zookeeper Client should re-reso...

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

    https://github.com/apache/zookeeper/pull/451#discussion_r186759876
  
    --- Diff: src/java/main/org/apache/zookeeper/client/StaticHostProvider.java ---
    @@ -111,9 +154,18 @@ public InetSocketAddress next(long spinDelay) {
                 lastIndex = 0;
             }
     
    -        return serverAddresses.get(currentIndex);
    +        InetSocketAddress curAddr = serverAddresses.get(currentIndex);
    +
    +        String curHostString = getHostString(curAddr);
    +        List<InetAddress> resolvedAddresses = new ArrayList<InetAddress>(Arrays.asList(this.resolver.getAllByName(curHostString)));
    +        if (resolvedAddresses.isEmpty()) {
    +            throw new UnknownHostException("No IP address returned for address: " + curHostString);
    --- End diff --
    
    Ok.


---

[GitHub] zookeeper pull request #451: ZOOKEEPER-2184: Zookeeper Client should re-reso...

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

    https://github.com/apache/zookeeper/pull/451


---

[GitHub] zookeeper pull request #451: ZOOKEEPER-2184: Zookeeper Client should re-reso...

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

    https://github.com/apache/zookeeper/pull/451#discussion_r186546855
  
    --- Diff: src/java/main/org/apache/zookeeper/client/HostProvider.java ---
    @@ -53,7 +54,7 @@
          * @param spinDelay
          *            Milliseconds to wait if all hosts have been tried once.
          */
    -    public InetSocketAddress next(long spinDelay);
    +    public InetSocketAddress next(long spinDelay) throws UnknownHostException;
    --- End diff --
    
    It feels odd to throw unknown host here because `next` is supposed to return a resolved host. The caller is asking for next resolved, not to resolve a given host name. Why do we need to throw it here and assuming we do need to throw something here, is there an exception that captures better the error you are trying to propagate?


---

[GitHub] zookeeper pull request #451: ZOOKEEPER-2184: Zookeeper Client should re-reso...

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

    https://github.com/apache/zookeeper/pull/451#discussion_r186591545
  
    --- Diff: src/java/main/org/apache/zookeeper/client/StaticHostProvider.java ---
    @@ -111,9 +154,18 @@ public InetSocketAddress next(long spinDelay) {
                 lastIndex = 0;
             }
     
    -        return serverAddresses.get(currentIndex);
    +        InetSocketAddress curAddr = serverAddresses.get(currentIndex);
    +
    +        String curHostString = getHostString(curAddr);
    +        List<InetAddress> resolvedAddresses = new ArrayList<InetAddress>(Arrays.asList(this.resolver.getAllByName(curHostString)));
    +        if (resolvedAddresses.isEmpty()) {
    +            throw new UnknownHostException("No IP address returned for address: " + curHostString);
    --- End diff --
    
    I believe the expectation - and the original behaviour- is to loop forever until the DNS problem is fixed. Even if we don't do it here, the caller will.
    
    I'd loop here infinitely. Also should keep an eye on spinDelays to avoid hammering the logic.


---

[GitHub] zookeeper issue #451: ZOOKEEPER-2184: Zookeeper Client should re-resolve hos...

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

    https://github.com/apache/zookeeper/pull/451
  
    @ijuma Sure, no problem.
    I'm waiting for some feedback from the community here and on the mailing list and I hope I can commit soon.


---

[GitHub] zookeeper pull request #451: ZOOKEEPER-2184: Zookeeper Client should re-reso...

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

    https://github.com/apache/zookeeper/pull/451#discussion_r168565039
  
    --- Diff: src/java/main/org/apache/zookeeper/client/StaticHostProvider.java ---
    @@ -18,6 +18,10 @@
     
     package org.apache.zookeeper.client;
     
    +import org.apache.yetus.audience.InterfaceAudience;
    --- End diff --
    
    nit: move this back


---

[GitHub] zookeeper pull request #451: ZOOKEEPER-2184: Zookeeper Client should re-reso...

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

    https://github.com/apache/zookeeper/pull/451#discussion_r163570637
  
    --- Diff: src/java/test/org/apache/zookeeper/client/StaticHostProviderTest.java ---
    @@ -117,8 +117,32 @@ public void testTwoInvalidHostAddresses() {
             list.add(new InetSocketAddress("a", 2181));
             list.add(new InetSocketAddress("b", 2181));
             new StaticHostProvider(list);
    +	}
    +
    +    @Test
    +    public void testReResolving() {
    +        byte size = 1;
    +        ArrayList<InetSocketAddress> list = new ArrayList<InetSocketAddress>(size);
    +
    +        // Test a hostname that resolves to multiple addresses
    --- End diff --
    
    Common domain names like facebook.com, google.com or apache.org don't use round-robin DNS anymore for some reason, so they don't resolve to multiple addresses. As a consequence this test doesn't validate the branch related to multiple addresses anymore unfortunately.
    
    Not sure how to address that, PowerMock would be the best to mock static `getAllByName()` method, but that would involve introducing a new test dependency.


---

[GitHub] zookeeper pull request #451: ZOOKEEPER-2184: Zookeeper Client should re-reso...

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

    https://github.com/apache/zookeeper/pull/451#discussion_r191034929
  
    --- Diff: src/java/main/org/apache/zookeeper/client/HostProvider.java ---
    @@ -53,7 +54,7 @@
          * @param spinDelay
          *            Milliseconds to wait if all hosts have been tried once.
          */
    -    public InetSocketAddress next(long spinDelay);
    +    public InetSocketAddress next(long spinDelay) throws UnknownHostException;
    --- End diff --
    
    >> In a nutshell the problem of unresolvable DNS name must be handled somewhere and here're the considerations
    
    I agree with this, however this can be done without throwing the UnknownHostException here. We can just return the curAddr in case of resolving failure, and since curAddr is not resolved, trying to connect to it on caller side will not succeed and caller can continue retry etc with existing logic. The primary concern here is change of a public API which might require clients (e.g. HBase) to recompile with ZK, so I prefer the approach that can achieve same goal (let caller handle retry logic) w/o changing the signature of a public API (alternatively we can throw an unchecked exception but semantically that sounds no better than UnknownHostException).
    
    So I am thinking the semantic of next() is:
    
    * If we can resolve an address, resolve it and return.
    * If we can't resolve an address, return an unresolved address which contain enough information (host, port etc) for caller to decide what to do.


---

[GitHub] zookeeper pull request #451: ZOOKEEPER-2184: Zookeeper Client should re-reso...

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

    https://github.com/apache/zookeeper/pull/451#discussion_r166476047
  
    --- Diff: src/java/test/org/apache/zookeeper/test/ReadOnlyModeTest.java ---
    @@ -239,13 +243,13 @@ public void testSessionEstablishment() throws Exception {
         public void testSeekForRwServer() throws Exception {
     
             // setup the logger to capture all logs
    -        Layout layout = Logger.getRootLogger().getAppender("CONSOLE")
    +        Layout layout = org.apache.log4j.Logger.getRootLogger().getAppender("CONSOLE")
    --- End diff --
    
    not going to push too hard on this, but I think you can just use log4j everywhere like in `QuorumPeerMainTest`


---

[GitHub] zookeeper pull request #451: ZOOKEEPER-2184: Zookeeper Client should re-reso...

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

    https://github.com/apache/zookeeper/pull/451#discussion_r167024767
  
    --- Diff: src/java/main/org/apache/zookeeper/client/StaticHostProvider.java ---
    @@ -58,48 +61,122 @@
         public StaticHostProvider(Collection<InetSocketAddress> serverAddresses) {
             for (InetSocketAddress address : serverAddresses) {
                 try {
    -                InetAddress ia = address.getAddress();
    -                InetAddress resolvedAddresses[] = InetAddress.getAllByName((ia != null) ? ia.getHostAddress() :
    -                        address.getHostName());
    +                InetAddress resolvedAddresses[] = InetAddress.getAllByName(getHostString(address));
                     for (InetAddress resolvedAddress : resolvedAddresses) {
    -                    // If hostName is null but the address is not, we can tell that
    -                    // the hostName is an literal IP address. Then we can set the host string as the hostname
    -                    // safely to avoid reverse DNS lookup.
    -                    // As far as i know, the only way to check if the hostName is null is use toString().
    -                    // Both the two implementations of InetAddress are final class, so we can trust the return value of
    -                    // the toString() method.
    -                    if (resolvedAddress.toString().startsWith("/")
    -                            && resolvedAddress.getAddress() != null) {
    -                        this.serverAddresses.add(
    -                                new InetSocketAddress(InetAddress.getByAddress(
    -                                        address.getHostName(),
    -                                        resolvedAddress.getAddress()),
    -                                        address.getPort()));
    -                    } else {
    -                        this.serverAddresses.add(new InetSocketAddress(resolvedAddress.getHostAddress(), address.getPort()));
    -                    }
    +                    this.serverAddresses.add(new InetSocketAddress(resolvedAddress, address.getPort()));
                     }
                 } catch (UnknownHostException e) {
                     LOG.error("Unable to connect to server: {}", address, e);
                 }
             }
    -        
    +
             if (this.serverAddresses.isEmpty()) {
                 throw new IllegalArgumentException(
                         "A HostProvider may not be empty!");
             }
             Collections.shuffle(this.serverAddresses);
         }
     
    +    /**
    +     * Evaluate to a hostname if one is available and otherwise it returns the
    +     * string representation of the IP address.
    +     *
    +     * In Java 7, we have a method getHostString, but earlier versions do not support it.
    +     * This method is to provide a replacement for InetSocketAddress.getHostString().
    +     *
    +     * @param addr
    +     * @return Hostname string of address parameter
    +     */
    +    private String getHostString(InetSocketAddress addr) {
    +        String hostString = "";
    +
    +        if (addr == null) {
    +            return hostString;
    +        }
    +        if (!addr.isUnresolved()) {
    +            InetAddress ia = addr.getAddress();
    +
    +            // If the string starts with '/', then it has no hostname
    +            // and we want to avoid the reverse lookup, so we return
    +            // the string representation of the address.
    +            if (ia.toString().startsWith("/")) {
    +                hostString = ia.getHostAddress();
    +            } else {
    +                hostString = addr.getHostName();
    +            }
    +        } else {
    +            // According to the Java 6 documentation, if the hostname is
    +            // unresolved, then the string before the colon is the hostname.
    +            String addrString = addr.toString();
    +            hostString = addrString.substring(0, addrString.lastIndexOf(':'));
    +        }
    +
    +        return hostString;
    +    }
    +
         public int size() {
             return serverAddresses.size();
         }
     
    +    // Counts the number of addresses added and removed during
    +    // the last call to next. Used mainly for test purposes.
    +    // See StasticHostProviderTest.
    +    private int nextAdded = 0;
    +    private int nextRemoved = 0;
    +
    +    public int getNextAdded() {
    +        return nextAdded;
    +    }
    +
    +    public int getNextRemoved() {
    +        return nextRemoved;
    +    }
    +
         public InetSocketAddress next(long spinDelay) {
    -        ++currentIndex;
    -        if (currentIndex == serverAddresses.size()) {
    -            currentIndex = 0;
    +        // Handle possible connection error by re-resolving hostname if possible
    +        if (!connectedSinceNext) {
    +            InetSocketAddress curAddr = serverAddresses.get(currentIndex);
    +            String curHostString = getHostString(curAddr);
    +            if (!curHostString.equals(curAddr.getAddress().getHostAddress())) {
    +                LOG.info("Resolving again hostname: {}", getHostString(curAddr));
    +                try {
    +                    int thePort = curAddr.getPort();
    +                    InetAddress resolvedAddresses[] = InetAddress.getAllByName(curHostString);
    +                    nextAdded = 0;
    +                    nextRemoved = 0;
    +                    if (resolvedAddresses.length == 1) {
    +                        serverAddresses.set(currentIndex, new InetSocketAddress(resolvedAddresses[0], thePort));
    +                        nextAdded = nextRemoved = 1;
    +                        LOG.debug("Newly resolved address: {}", resolvedAddresses[0]);
    +                    } else {
    +                        int i = 0;
    +                        while (i < serverAddresses.size()) {
    +                            if (getHostString(serverAddresses.get(i)).equals(curHostString) &&
    --- End diff --
    
    The key is the hoststring which is like `issues.apache.org` and could have multiple InetAddresses which are the values of the key in the map. If I shuffle only the keys, I can iterate on only the keys and not sure how to choose from the addresses.


---

[GitHub] zookeeper pull request #451: ZOOKEEPER-2184: Zookeeper Client should re-reso...

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

    https://github.com/apache/zookeeper/pull/451#discussion_r168763559
  
    --- Diff: src/java/test/org/apache/zookeeper/test/ReadOnlyModeTest.java ---
    @@ -239,13 +243,13 @@ public void testSessionEstablishment() throws Exception {
         public void testSeekForRwServer() throws Exception {
     
             // setup the logger to capture all logs
    -        Layout layout = Logger.getRootLogger().getAppender("CONSOLE")
    +        Layout layout = org.apache.log4j.Logger.getRootLogger().getAppender("CONSOLE")
    --- End diff --
    
    Makes sense. I revert the changes.


---

[GitHub] zookeeper issue #451: ZOOKEEPER-2184: Zookeeper Client should re-resolve hos...

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

    https://github.com/apache/zookeeper/pull/451
  
    If there are no other concerns, it would be great for this to be merged.


---

[GitHub] zookeeper pull request #451: ZOOKEEPER-2184: Zookeeper Client should re-reso...

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

    https://github.com/apache/zookeeper/pull/451#discussion_r165525652
  
    --- Diff: src/java/test/org/apache/zookeeper/test/ReadOnlyModeTest.java ---
    @@ -239,13 +243,13 @@ public void testSessionEstablishment() throws Exception {
         public void testSeekForRwServer() throws Exception {
     
             // setup the logger to capture all logs
    -        Layout layout = Logger.getRootLogger().getAppender("CONSOLE")
    +        Layout layout = org.apache.log4j.Logger.getRootLogger().getAppender("CONSOLE")
    --- End diff --
    
    why is this necessary?


---

[GitHub] zookeeper pull request #451: ZOOKEEPER-2184: Zookeeper Client should re-reso...

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

    https://github.com/apache/zookeeper/pull/451#discussion_r186694289
  
    --- Diff: src/java/main/org/apache/zookeeper/client/StaticHostProvider.java ---
    @@ -30,76 +31,118 @@
     import org.slf4j.Logger;
     import org.slf4j.LoggerFactory;
     
    -
     /**
      * Most simple HostProvider, resolves only on instantiation.
    - * 
    + *
      */
     @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);
     
    -    private final List<InetSocketAddress> serverAddresses = new ArrayList<InetSocketAddress>(
    -            5);
    +    private final List<InetSocketAddress> serverAddresses = new ArrayList<InetSocketAddress>(5);
     
         private int lastIndex = -1;
     
         private int currentIndex = -1;
     
    +    private Resolver resolver;
    +
         /**
          * Constructs a SimpleHostSet.
    -     * 
    +     *
          * @param serverAddresses
          *            possibly unresolved ZooKeeper server addresses
          * @throws IllegalArgumentException
          *             if serverAddresses is empty or resolves to an empty list
          */
         public StaticHostProvider(Collection<InetSocketAddress> serverAddresses) {
    -        for (InetSocketAddress address : serverAddresses) {
    -            try {
    -                InetAddress ia = address.getAddress();
    -                InetAddress resolvedAddresses[] = InetAddress.getAllByName((ia != null) ? ia.getHostAddress() :
    -                        address.getHostName());
    -                for (InetAddress resolvedAddress : resolvedAddresses) {
    -                    // If hostName is null but the address is not, we can tell that
    -                    // the hostName is an literal IP address. Then we can set the host string as the hostname
    -                    // safely to avoid reverse DNS lookup.
    -                    // As far as i know, the only way to check if the hostName is null is use toString().
    -                    // Both the two implementations of InetAddress are final class, so we can trust the return value of
    -                    // the toString() method.
    -                    if (resolvedAddress.toString().startsWith("/")
    -                            && resolvedAddress.getAddress() != null) {
    -                        this.serverAddresses.add(
    -                                new InetSocketAddress(InetAddress.getByAddress(
    -                                        address.getHostName(),
    -                                        resolvedAddress.getAddress()),
    -                                        address.getPort()));
    -                    } else {
    -                        this.serverAddresses.add(new InetSocketAddress(resolvedAddress.getHostAddress(), address.getPort()));
    -                    }
    -                }
    -            } catch (UnknownHostException e) {
    -                LOG.error("Unable to connect to server: {}", address, e);
    +        this.resolver = new Resolver() {
    +            @Override
    +            public InetAddress[] getAllByName(String name) throws UnknownHostException {
    +                return InetAddress.getAllByName(name);
                 }
    -        }
    -        
    -        if (this.serverAddresses.isEmpty()) {
    +        };
    +        init(serverAddresses);
    +    }
    +
    +    /**
    +     * 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
    +     * @throws IllegalArgumentException
    +     *             if serverAddresses is empty or resolves to an empty list
    +     */
    +    public StaticHostProvider(Collection<InetSocketAddress> serverAddresses, Resolver resolver) {
    +        this.resolver = resolver;
    +        init(serverAddresses);
    +    }
    +
    +    /**
    +     * Common init method for all constructors.
    +     * Resolve all unresolved server addresses, put them in a list and shuffle.
    +     */
    +    private void init(Collection<InetSocketAddress> serverAddresses) {
    +        if (serverAddresses.isEmpty()) {
                 throw new IllegalArgumentException(
                         "A HostProvider may not be empty!");
             }
    +
    +        this.serverAddresses.addAll(serverAddresses);
             Collections.shuffle(this.serverAddresses);
         }
     
    +    /**
    +     * Evaluate to a hostname if one is available and otherwise it returns the
    +     * string representation of the IP address.
    +     *
    +     * In Java 7, we have a method getHostString, but earlier versions do not support it.
    +     * This method is to provide a replacement for InetSocketAddress.getHostString().
    +     *
    +     * @param addr
    +     * @return Hostname string of address parameter
    +     */
    +    private String getHostString(InetSocketAddress addr) {
    --- End diff --
    
     As ZK JVM version is jdk7, there is still a need for this method?


---

[GitHub] zookeeper pull request #451: ZOOKEEPER-2184: Zookeeper Client should re-reso...

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

    https://github.com/apache/zookeeper/pull/451#discussion_r186734788
  
    --- Diff: src/java/main/org/apache/zookeeper/client/StaticHostProvider.java ---
    @@ -111,9 +154,18 @@ public InetSocketAddress next(long spinDelay) {
                 lastIndex = 0;
             }
     
    -        return serverAddresses.get(currentIndex);
    +        InetSocketAddress curAddr = serverAddresses.get(currentIndex);
    +
    +        String curHostString = getHostString(curAddr);
    +        List<InetAddress> resolvedAddresses = new ArrayList<InetAddress>(Arrays.asList(this.resolver.getAllByName(curHostString)));
    +        if (resolvedAddresses.isEmpty()) {
    +            throw new UnknownHostException("No IP address returned for address: " + curHostString);
    --- End diff --
    
    > We need a way to break the loop in the case the client closes, though.
    
    That's actually a good reason for _not_ dealing with the error here. Because the caller - ClientCnxn - is be able to detect client closes, but StatisHostProvider is not.


---

[GitHub] zookeeper pull request #451: ZOOKEEPER-2184: Zookeeper Client should re-reso...

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

    https://github.com/apache/zookeeper/pull/451#discussion_r191087859
  
    --- Diff: src/java/main/org/apache/zookeeper/client/HostProvider.java ---
    @@ -53,7 +54,7 @@
          * @param spinDelay
          *            Milliseconds to wait if all hosts have been tried once.
          */
    -    public InetSocketAddress next(long spinDelay);
    +    public InetSocketAddress next(long spinDelay) throws UnknownHostException;
    --- End diff --
    
    That's actually a great idea. I've tried that and `socket.connect()` will throw an IOException eventually of the address is not resolved. Which means basically that the same retry logic applies in the caller, it's just not next() that catches the error. I'll update the patch.
    
    Once this one is finally merged, I'll take a look at 3.5/master branches, because I think we should be consistent with this across the board.



---

[GitHub] zookeeper pull request #451: ZOOKEEPER-2184: Zookeeper Client should re-reso...

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

    https://github.com/apache/zookeeper/pull/451#discussion_r191242849
  
    --- Diff: src/java/main/org/apache/zookeeper/client/HostProvider.java ---
    @@ -21,6 +21,7 @@
     import org.apache.yetus.audience.InterfaceAudience;
     
     import java.net.InetSocketAddress;
    +import java.net.UnknownHostException;
    --- End diff --
    
    This import is not used, we can remove it.


---

[GitHub] zookeeper pull request #451: ZOOKEEPER-2184: Zookeeper Client should re-reso...

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

    https://github.com/apache/zookeeper/pull/451#discussion_r191243666
  
    --- Diff: src/java/test/org/apache/zookeeper/test/ClientPortBindTest.java ---
    @@ -102,9 +102,9 @@ public void testBindByAddress() throws Exception {
             startSignal = new CountDownLatch(1);
             ZooKeeper zk = new ZooKeeper(HOSTPORT, CONNECTION_TIMEOUT, this);
             try {
    -            startSignal.await(CONNECTION_TIMEOUT,
    --- End diff --
    
    The change in this file sounds not related to this patch. Usually we tend to limit changes only relevant to a patch. Similar for the change in ReadOnlyModeTest file. It's fine in this case as both changes are small and are good improvements.


---

[GitHub] zookeeper pull request #451: ZOOKEEPER-2184: Zookeeper Client should re-reso...

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

    https://github.com/apache/zookeeper/pull/451#discussion_r165665505
  
    --- Diff: src/java/main/org/apache/zookeeper/client/StaticHostProvider.java ---
    @@ -91,15 +79,106 @@ public StaticHostProvider(Collection<InetSocketAddress> serverAddresses) {
             Collections.shuffle(this.serverAddresses);
         }
     
    +    /**
    +     * Evaluate to a hostname if one is available and otherwise it returns the
    +     * string representation of the IP address.
    +     *
    +     * In Java 7, we have a method getHostString, but earlier versions do not support it.
    +     * This method is to provide a replacement for InetSocketAddress.getHostString().
    +     *
    +     * @param addr
    +     * @return Hostname string of address parameter
    +     */
    +    private String getHostString(InetSocketAddress addr) {
    +        String hostString = "";
    +
    +        if (addr == null) {
    +            return hostString;
    +        }
    +        if (!addr.isUnresolved()) {
    +            InetAddress ia = addr.getAddress();
    +
    +            // If the string starts with '/', then it has no hostname
    +            // and we want to avoid the reverse lookup, so we return
    +            // the string representation of the address.
    +            if (ia.toString().startsWith("/")) {
    +                hostString = ia.getHostAddress();
    +            } else {
    +                hostString = addr.getHostName();
    +            }
    +        } else {
    +            // According to the Java 6 documentation, if the hostname is
    +            // unresolved, then the string before the colon is the hostname.
    +            String addrString = addr.toString();
    +            hostString = addrString.substring(0, addrString.lastIndexOf(':'));
    +        }
    +
    +        return hostString;
    +    }
    +
         public int size() {
             return serverAddresses.size();
         }
     
    +    // Counts the number of addresses added and removed during
    +    // the last call to next. Used mainly for test purposes.
    +    // See StasticHostProviderTest.
    +    private int nextAdded = 0;
    +    private int nextRemoved = 0;
    +
    +    int getNextAdded() {
    +        return nextAdded;
    +    }
    +
    +    int getNextRemoved() {
    +        return nextRemoved;
    +    }
    +
         public InetSocketAddress next(long spinDelay) {
    -        ++currentIndex;
    -        if (currentIndex == serverAddresses.size()) {
    -            currentIndex = 0;
    +        // Handle possible connection error by re-resolving hostname if possible
    +        if (!connectedSinceNext) {
    --- End diff --
    
    It should try to re-resolve whenever the client is unable to connect to a server (connectedSinceNext == false). 
    
    @fpj gives a good explanation in the original Jira:
    https://issues.apache.org/jira/browse/ZOOKEEPER-2184?focusedCommentId=15873730&page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel#comment-15873730
    
    > I haven't had much time to work on this issue, but here is my current assessment.
    
    > This issue seemed easy to fix at first, but it is fairly fundamental with respect to how we resolve host names. Currently, we resolve host names when we start a client and never resolve it again. This is the cause of the problem reported in the issue because in the scenario described, the zookeeper container is re-started and changes addresses, which prevents the client from connecting to the zookeeper server.
    
    > The proposed patch here tries to re-resolve the hostname every time the client fails to connect to the resolved address. It kind of works, but it makes StaticHostProvider a bit messy because the expectation with the current wiring is that we won't have to resolve again.
    
    > The ideal situation for the problematic scenario is that we resolve the host name every time we try to connect to a server, but that would be a fairly fundamental change to how we resolve addresses in ZooKeeper.
    
    > I was also looking at the C client and it might get a bit messy too there because I don't think we currently keep the association between the host name and the resolved address, so we don't really know what to resolve again. It might be possible to do it via the canonical name in getaddrinfo, but I'm not sure how that works with windows.
    
    > One specific proposal to avoid having clients never finding a server ever again without deep changes to the current wiring is to resolve again everything in the case the client tries all and none succeeds. That would be a fairly straightforward change to both Java and C client, but it would not resolve addresses again in the case the a strict subset has changed addresses and at least one server is reachable.


---

[GitHub] zookeeper pull request #451: ZOOKEEPER-2184: Zookeeper Client should re-reso...

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

    https://github.com/apache/zookeeper/pull/451#discussion_r166341540
  
    --- Diff: src/java/main/org/apache/zookeeper/client/StaticHostProvider.java ---
    @@ -58,48 +61,122 @@
         public StaticHostProvider(Collection<InetSocketAddress> serverAddresses) {
             for (InetSocketAddress address : serverAddresses) {
                 try {
    -                InetAddress ia = address.getAddress();
    -                InetAddress resolvedAddresses[] = InetAddress.getAllByName((ia != null) ? ia.getHostAddress() :
    -                        address.getHostName());
    +                InetAddress resolvedAddresses[] = InetAddress.getAllByName(getHostString(address));
                     for (InetAddress resolvedAddress : resolvedAddresses) {
    -                    // If hostName is null but the address is not, we can tell that
    -                    // the hostName is an literal IP address. Then we can set the host string as the hostname
    -                    // safely to avoid reverse DNS lookup.
    -                    // As far as i know, the only way to check if the hostName is null is use toString().
    -                    // Both the two implementations of InetAddress are final class, so we can trust the return value of
    -                    // the toString() method.
    -                    if (resolvedAddress.toString().startsWith("/")
    -                            && resolvedAddress.getAddress() != null) {
    -                        this.serverAddresses.add(
    -                                new InetSocketAddress(InetAddress.getByAddress(
    -                                        address.getHostName(),
    -                                        resolvedAddress.getAddress()),
    -                                        address.getPort()));
    -                    } else {
    -                        this.serverAddresses.add(new InetSocketAddress(resolvedAddress.getHostAddress(), address.getPort()));
    -                    }
    +                    this.serverAddresses.add(new InetSocketAddress(resolvedAddress, address.getPort()));
                     }
                 } catch (UnknownHostException e) {
                     LOG.error("Unable to connect to server: {}", address, e);
                 }
             }
    -        
    +
             if (this.serverAddresses.isEmpty()) {
                 throw new IllegalArgumentException(
                         "A HostProvider may not be empty!");
             }
             Collections.shuffle(this.serverAddresses);
         }
     
    +    /**
    +     * Evaluate to a hostname if one is available and otherwise it returns the
    +     * string representation of the IP address.
    +     *
    +     * In Java 7, we have a method getHostString, but earlier versions do not support it.
    +     * This method is to provide a replacement for InetSocketAddress.getHostString().
    +     *
    +     * @param addr
    +     * @return Hostname string of address parameter
    +     */
    +    private String getHostString(InetSocketAddress addr) {
    +        String hostString = "";
    +
    +        if (addr == null) {
    +            return hostString;
    +        }
    +        if (!addr.isUnresolved()) {
    +            InetAddress ia = addr.getAddress();
    +
    +            // If the string starts with '/', then it has no hostname
    +            // and we want to avoid the reverse lookup, so we return
    +            // the string representation of the address.
    +            if (ia.toString().startsWith("/")) {
    +                hostString = ia.getHostAddress();
    +            } else {
    +                hostString = addr.getHostName();
    +            }
    +        } else {
    +            // According to the Java 6 documentation, if the hostname is
    +            // unresolved, then the string before the colon is the hostname.
    +            String addrString = addr.toString();
    +            hostString = addrString.substring(0, addrString.lastIndexOf(':'));
    +        }
    +
    +        return hostString;
    +    }
    +
         public int size() {
             return serverAddresses.size();
         }
     
    +    // Counts the number of addresses added and removed during
    +    // the last call to next. Used mainly for test purposes.
    +    // See StasticHostProviderTest.
    +    private int nextAdded = 0;
    +    private int nextRemoved = 0;
    +
    +    public int getNextAdded() {
    +        return nextAdded;
    +    }
    +
    +    public int getNextRemoved() {
    +        return nextRemoved;
    +    }
    +
         public InetSocketAddress next(long spinDelay) {
    -        ++currentIndex;
    -        if (currentIndex == serverAddresses.size()) {
    -            currentIndex = 0;
    +        // Handle possible connection error by re-resolving hostname if possible
    +        if (!connectedSinceNext) {
    +            InetSocketAddress curAddr = serverAddresses.get(currentIndex);
    +            String curHostString = getHostString(curAddr);
    +            if (!curHostString.equals(curAddr.getAddress().getHostAddress())) {
    +                LOG.info("Resolving again hostname: {}", getHostString(curAddr));
    +                try {
    +                    int thePort = curAddr.getPort();
    +                    InetAddress resolvedAddresses[] = InetAddress.getAllByName(curHostString);
    +                    nextAdded = 0;
    +                    nextRemoved = 0;
    +                    if (resolvedAddresses.length == 1) {
    +                        serverAddresses.set(currentIndex, new InetSocketAddress(resolvedAddresses[0], thePort));
    +                        nextAdded = nextRemoved = 1;
    +                        LOG.debug("Newly resolved address: {}", resolvedAddresses[0]);
    +                    } else {
    +                        int i = 0;
    +                        while (i < serverAddresses.size()) {
    +                            if (getHostString(serverAddresses.get(i)).equals(curHostString) &&
    --- End diff --
    
    It seems that this change makes things a little a bit even more complicated, because `next()` method needs to have a list of IP addresses to iterate on. Maintenance is easier with a Map, but I need to find a way to use it as a list.


---

[GitHub] zookeeper issue #451: ZOOKEEPER-2184: Zookeeper Client should re-resolve hos...

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

    https://github.com/apache/zookeeper/pull/451
  
    There are also some initial comments about JVM DNS caching
    
    >> Re-resolving at StaticHostProvider level may not be sufficient as InetAddress.getAllByName(String host) itself uses a Java-level cache inside InetAddress and turns to name service (e.g. DNS) only if the host could not be found in the Java-level cache.
    
    I think it's a good point that we brought this up - and I think we can't do much here as there are multiple level of DNS caching not just in JVM but also in OS, switches, etc, so the best we could do here is to make sure we re-resolve address when necessary, as this pull request is doing, and meanwhile let user know this might not work in the end due to multiple level of DNS caching. In that case, I think some links to the Oracle documents about JVM DNS caching would be helpful so users don't blame ZooKeeper if things don't work, but I think we can do that documentation update separately after this pull request is merged.


---

[GitHub] zookeeper pull request #451: ZOOKEEPER-2184: Zookeeper Client should re-reso...

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

    https://github.com/apache/zookeeper/pull/451#discussion_r186689865
  
    --- Diff: src/java/main/org/apache/zookeeper/client/HostProvider.java ---
    @@ -53,7 +54,7 @@
          * @param spinDelay
          *            Milliseconds to wait if all hosts have been tried once.
          */
    -    public InetSocketAddress next(long spinDelay);
    +    public InetSocketAddress next(long spinDelay) throws UnknownHostException;
    --- End diff --
    
    My preference is the following:
    
    1- That we do not make this API change. It is not necessary to solve the problem of this issue.
    2- That we discuss separately whether we want to change the behaviour of the `next()` in the `HostProvider` interface.


---

[GitHub] zookeeper pull request #451: ZOOKEEPER-2184: Zookeeper Client should re-reso...

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

    https://github.com/apache/zookeeper/pull/451#discussion_r168568596
  
    --- Diff: src/java/main/org/apache/zookeeper/client/StaticHostProvider.java ---
    @@ -47,59 +51,169 @@
     
         private int currentIndex = -1;
     
    +    // Don't re-resolve on first next() call
    +    private boolean connectedSinceNext = true;
    +
    +    private Resolver resolver;
    +
         /**
          * Constructs a SimpleHostSet.
    -     * 
    +     *
          * @param serverAddresses
          *            possibly unresolved ZooKeeper server addresses
          * @throws IllegalArgumentException
          *             if serverAddresses is empty or resolves to an empty list
          */
         public StaticHostProvider(Collection<InetSocketAddress> serverAddresses) {
    +        this.resolver = new Resolver() {
    +            @Override
    +            public InetAddress[] getAllByName(String name) throws UnknownHostException {
    +                return InetAddress.getAllByName(name);
    +            }
    +        };
    +        init(serverAddresses);
    +    }
    +
    +    /**
    +     * 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
    +     * @throws IllegalArgumentException
    +     *             if serverAddresses is empty or resolves to an empty list
    +     */
    +    public StaticHostProvider(Collection<InetSocketAddress> serverAddresses, Resolver resolver) {
    +        this.resolver = resolver;
    +        init(serverAddresses);
    +    }
    +
    +    /**
    +     * Common init method for all constructors.
    +     * Resolve all unresolved server addresses, put them in a list and shuffle.
    +     */
    +    private void init(Collection<InetSocketAddress> serverAddresses) {
             for (InetSocketAddress address : serverAddresses) {
                 try {
    -                InetAddress ia = address.getAddress();
    -                InetAddress resolvedAddresses[] = InetAddress.getAllByName((ia != null) ? ia.getHostAddress() :
    -                        address.getHostName());
    +                InetAddress resolvedAddresses[] = this.resolver.getAllByName(getHostString(address));
                     for (InetAddress resolvedAddress : resolvedAddresses) {
    -                    // If hostName is null but the address is not, we can tell that
    -                    // the hostName is an literal IP address. Then we can set the host string as the hostname
    -                    // safely to avoid reverse DNS lookup.
    -                    // As far as i know, the only way to check if the hostName is null is use toString().
    -                    // Both the two implementations of InetAddress are final class, so we can trust the return value of
    -                    // the toString() method.
    -                    if (resolvedAddress.toString().startsWith("/")
    -                            && resolvedAddress.getAddress() != null) {
    -                        this.serverAddresses.add(
    -                                new InetSocketAddress(InetAddress.getByAddress(
    -                                        address.getHostName(),
    -                                        resolvedAddress.getAddress()),
    -                                        address.getPort()));
    -                    } else {
    -                        this.serverAddresses.add(new InetSocketAddress(resolvedAddress.getHostAddress(), address.getPort()));
    -                    }
    +                    this.serverAddresses.add(new InetSocketAddress(resolvedAddress, address.getPort()));
                     }
                 } catch (UnknownHostException e) {
                     LOG.error("Unable to connect to server: {}", address, e);
                 }
             }
    -        
    +
             if (this.serverAddresses.isEmpty()) {
                 throw new IllegalArgumentException(
                         "A HostProvider may not be empty!");
             }
    +
             Collections.shuffle(this.serverAddresses);
         }
     
    +    /**
    +     * Evaluate to a hostname if one is available and otherwise it returns the
    +     * string representation of the IP address.
    +     *
    +     * In Java 7, we have a method getHostString, but earlier versions do not support it.
    +     * This method is to provide a replacement for InetSocketAddress.getHostString().
    +     *
    +     * @param addr
    +     * @return Hostname string of address parameter
    +     */
    +    private String getHostString(InetSocketAddress addr) {
    +        String hostString = "";
    +
    +        if (addr == null) {
    +            return hostString;
    +        }
    +        if (!addr.isUnresolved()) {
    +            InetAddress ia = addr.getAddress();
    +
    +            // If the string starts with '/', then it has no hostname
    +            // and we want to avoid the reverse lookup, so we return
    +            // the string representation of the address.
    +            if (ia.toString().startsWith("/")) {
    +                hostString = ia.getHostAddress();
    +            } else {
    +                hostString = addr.getHostName();
    +            }
    +        } else {
    +            // According to the Java 6 documentation, if the hostname is
    +            // unresolved, then the string before the colon is the hostname.
    +            String addrString = addr.toString();
    +            hostString = addrString.substring(0, addrString.lastIndexOf(':'));
    +        }
    +
    +        return hostString;
    +    }
    +
         public int size() {
             return serverAddresses.size();
         }
     
    +    // Counts the number of addresses added and removed during
    +    // the last call to next. Used mainly for test purposes.
    +    // See StasticHostProviderTest.
    +    private int nextAdded = 0;
    +    private int nextRemoved = 0;
    +
    +    public int getNextAdded() {
    +        return nextAdded;
    +    }
    +
    +    public int getNextRemoved() {
    +        return nextRemoved;
    +    }
    +
         public InetSocketAddress next(long spinDelay) {
    -        ++currentIndex;
    -        if (currentIndex == serverAddresses.size()) {
    -            currentIndex = 0;
    +        // Handle possible connection error by re-resolving hostname if possible
    +        if (!connectedSinceNext) {
    +            InetSocketAddress curAddr = serverAddresses.get(currentIndex);
    +            String curHostString = getHostString(curAddr);
    +            if (!curHostString.equals(curAddr.getAddress().getHostAddress())) {
    +                LOG.info("Resolving again hostname: {}", getHostString(curAddr));
    --- End diff --
    
    can we save a call to `getHostString` by using `curHostString`?


---

[GitHub] zookeeper pull request #451: ZOOKEEPER-2184: Zookeeper Client should re-reso...

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

    https://github.com/apache/zookeeper/pull/451#discussion_r168576884
  
    --- Diff: src/java/test/org/apache/zookeeper/test/StaticHostProviderTest.java ---
    @@ -119,6 +120,68 @@ public void testTwoInvalidHostAddresses() {
             new StaticHostProvider(list);
         }
     
    +    @Test
    +    public void testReResolvingSingle() {
    +        byte size = 1;
    +        ArrayList<InetSocketAddress> list = new ArrayList<InetSocketAddress>(size);
    +
    +        // Test a hostname that resolves to a single address
    +        list.clear();
    --- End diff --
    
    do we need this?


---

[GitHub] zookeeper pull request #451: ZOOKEEPER-2184: Zookeeper Client should re-reso...

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

    https://github.com/apache/zookeeper/pull/451#discussion_r166328583
  
    --- Diff: src/java/main/org/apache/zookeeper/client/StaticHostProvider.java ---
    @@ -58,48 +61,122 @@
         public StaticHostProvider(Collection<InetSocketAddress> serverAddresses) {
             for (InetSocketAddress address : serverAddresses) {
                 try {
    -                InetAddress ia = address.getAddress();
    -                InetAddress resolvedAddresses[] = InetAddress.getAllByName((ia != null) ? ia.getHostAddress() :
    -                        address.getHostName());
    +                InetAddress resolvedAddresses[] = InetAddress.getAllByName(getHostString(address));
                     for (InetAddress resolvedAddress : resolvedAddresses) {
    -                    // If hostName is null but the address is not, we can tell that
    -                    // the hostName is an literal IP address. Then we can set the host string as the hostname
    -                    // safely to avoid reverse DNS lookup.
    -                    // As far as i know, the only way to check if the hostName is null is use toString().
    -                    // Both the two implementations of InetAddress are final class, so we can trust the return value of
    -                    // the toString() method.
    -                    if (resolvedAddress.toString().startsWith("/")
    -                            && resolvedAddress.getAddress() != null) {
    -                        this.serverAddresses.add(
    -                                new InetSocketAddress(InetAddress.getByAddress(
    -                                        address.getHostName(),
    -                                        resolvedAddress.getAddress()),
    -                                        address.getPort()));
    -                    } else {
    -                        this.serverAddresses.add(new InetSocketAddress(resolvedAddress.getHostAddress(), address.getPort()));
    -                    }
    +                    this.serverAddresses.add(new InetSocketAddress(resolvedAddress, address.getPort()));
                     }
                 } catch (UnknownHostException e) {
                     LOG.error("Unable to connect to server: {}", address, e);
                 }
             }
    -        
    +
             if (this.serverAddresses.isEmpty()) {
                 throw new IllegalArgumentException(
                         "A HostProvider may not be empty!");
             }
             Collections.shuffle(this.serverAddresses);
         }
     
    +    /**
    +     * Evaluate to a hostname if one is available and otherwise it returns the
    +     * string representation of the IP address.
    +     *
    +     * In Java 7, we have a method getHostString, but earlier versions do not support it.
    +     * This method is to provide a replacement for InetSocketAddress.getHostString().
    +     *
    +     * @param addr
    +     * @return Hostname string of address parameter
    +     */
    +    private String getHostString(InetSocketAddress addr) {
    +        String hostString = "";
    +
    +        if (addr == null) {
    +            return hostString;
    +        }
    +        if (!addr.isUnresolved()) {
    +            InetAddress ia = addr.getAddress();
    +
    +            // If the string starts with '/', then it has no hostname
    +            // and we want to avoid the reverse lookup, so we return
    +            // the string representation of the address.
    +            if (ia.toString().startsWith("/")) {
    +                hostString = ia.getHostAddress();
    +            } else {
    +                hostString = addr.getHostName();
    +            }
    +        } else {
    +            // According to the Java 6 documentation, if the hostname is
    +            // unresolved, then the string before the colon is the hostname.
    +            String addrString = addr.toString();
    +            hostString = addrString.substring(0, addrString.lastIndexOf(':'));
    +        }
    +
    +        return hostString;
    +    }
    +
         public int size() {
             return serverAddresses.size();
         }
     
    +    // Counts the number of addresses added and removed during
    +    // the last call to next. Used mainly for test purposes.
    +    // See StasticHostProviderTest.
    +    private int nextAdded = 0;
    +    private int nextRemoved = 0;
    +
    +    public int getNextAdded() {
    +        return nextAdded;
    +    }
    +
    +    public int getNextRemoved() {
    +        return nextRemoved;
    +    }
    +
         public InetSocketAddress next(long spinDelay) {
    -        ++currentIndex;
    -        if (currentIndex == serverAddresses.size()) {
    -            currentIndex = 0;
    +        // Handle possible connection error by re-resolving hostname if possible
    +        if (!connectedSinceNext) {
    +            InetSocketAddress curAddr = serverAddresses.get(currentIndex);
    +            String curHostString = getHostString(curAddr);
    +            if (!curHostString.equals(curAddr.getAddress().getHostAddress())) {
    +                LOG.info("Resolving again hostname: {}", getHostString(curAddr));
    +                try {
    +                    int thePort = curAddr.getPort();
    +                    InetAddress resolvedAddresses[] = InetAddress.getAllByName(curHostString);
    +                    nextAdded = 0;
    +                    nextRemoved = 0;
    +                    if (resolvedAddresses.length == 1) {
    --- End diff --
    
    That's actually a very good point. I'm looking into that you suggested below (using a Map instead of Array) which will probably solve this issue too.


---

[GitHub] zookeeper pull request #451: ZOOKEEPER-2184: Zookeeper Client should re-reso...

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

    https://github.com/apache/zookeeper/pull/451#discussion_r165525113
  
    --- Diff: src/java/test/org/apache/zookeeper/client/StaticHostProviderTest.java ---
    @@ -16,7 +16,7 @@
      * limitations under the License.
      */
     
    -package org.apache.zookeeper.test;
    +package org.apache.zookeeper.client;
    --- End diff --
    
    this doesn't look right


---

[GitHub] zookeeper pull request #451: ZOOKEEPER-2184: Zookeeper Client should re-reso...

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

    https://github.com/apache/zookeeper/pull/451#discussion_r166670125
  
    --- Diff: src/java/main/org/apache/zookeeper/client/StaticHostProvider.java ---
    @@ -58,48 +61,122 @@
         public StaticHostProvider(Collection<InetSocketAddress> serverAddresses) {
             for (InetSocketAddress address : serverAddresses) {
                 try {
    -                InetAddress ia = address.getAddress();
    -                InetAddress resolvedAddresses[] = InetAddress.getAllByName((ia != null) ? ia.getHostAddress() :
    -                        address.getHostName());
    +                InetAddress resolvedAddresses[] = InetAddress.getAllByName(getHostString(address));
                     for (InetAddress resolvedAddress : resolvedAddresses) {
    -                    // If hostName is null but the address is not, we can tell that
    -                    // the hostName is an literal IP address. Then we can set the host string as the hostname
    -                    // safely to avoid reverse DNS lookup.
    -                    // As far as i know, the only way to check if the hostName is null is use toString().
    -                    // Both the two implementations of InetAddress are final class, so we can trust the return value of
    -                    // the toString() method.
    -                    if (resolvedAddress.toString().startsWith("/")
    -                            && resolvedAddress.getAddress() != null) {
    -                        this.serverAddresses.add(
    -                                new InetSocketAddress(InetAddress.getByAddress(
    -                                        address.getHostName(),
    -                                        resolvedAddress.getAddress()),
    -                                        address.getPort()));
    -                    } else {
    -                        this.serverAddresses.add(new InetSocketAddress(resolvedAddress.getHostAddress(), address.getPort()));
    -                    }
    +                    this.serverAddresses.add(new InetSocketAddress(resolvedAddress, address.getPort()));
                     }
                 } catch (UnknownHostException e) {
                     LOG.error("Unable to connect to server: {}", address, e);
                 }
             }
    -        
    +
             if (this.serverAddresses.isEmpty()) {
                 throw new IllegalArgumentException(
                         "A HostProvider may not be empty!");
             }
             Collections.shuffle(this.serverAddresses);
         }
     
    +    /**
    +     * Evaluate to a hostname if one is available and otherwise it returns the
    +     * string representation of the IP address.
    +     *
    +     * In Java 7, we have a method getHostString, but earlier versions do not support it.
    +     * This method is to provide a replacement for InetSocketAddress.getHostString().
    +     *
    +     * @param addr
    +     * @return Hostname string of address parameter
    +     */
    +    private String getHostString(InetSocketAddress addr) {
    +        String hostString = "";
    +
    +        if (addr == null) {
    +            return hostString;
    +        }
    +        if (!addr.isUnresolved()) {
    +            InetAddress ia = addr.getAddress();
    +
    +            // If the string starts with '/', then it has no hostname
    +            // and we want to avoid the reverse lookup, so we return
    +            // the string representation of the address.
    +            if (ia.toString().startsWith("/")) {
    +                hostString = ia.getHostAddress();
    +            } else {
    +                hostString = addr.getHostName();
    +            }
    +        } else {
    +            // According to the Java 6 documentation, if the hostname is
    +            // unresolved, then the string before the colon is the hostname.
    +            String addrString = addr.toString();
    +            hostString = addrString.substring(0, addrString.lastIndexOf(':'));
    +        }
    +
    +        return hostString;
    +    }
    +
         public int size() {
             return serverAddresses.size();
         }
     
    +    // Counts the number of addresses added and removed during
    +    // the last call to next. Used mainly for test purposes.
    +    // See StasticHostProviderTest.
    +    private int nextAdded = 0;
    +    private int nextRemoved = 0;
    +
    +    public int getNextAdded() {
    +        return nextAdded;
    +    }
    +
    +    public int getNextRemoved() {
    +        return nextRemoved;
    +    }
    +
         public InetSocketAddress next(long spinDelay) {
    -        ++currentIndex;
    -        if (currentIndex == serverAddresses.size()) {
    -            currentIndex = 0;
    +        // Handle possible connection error by re-resolving hostname if possible
    +        if (!connectedSinceNext) {
    +            InetSocketAddress curAddr = serverAddresses.get(currentIndex);
    +            String curHostString = getHostString(curAddr);
    +            if (!curHostString.equals(curAddr.getAddress().getHostAddress())) {
    +                LOG.info("Resolving again hostname: {}", getHostString(curAddr));
    +                try {
    +                    int thePort = curAddr.getPort();
    +                    InetAddress resolvedAddresses[] = InetAddress.getAllByName(curHostString);
    +                    nextAdded = 0;
    +                    nextRemoved = 0;
    +                    if (resolvedAddresses.length == 1) {
    +                        serverAddresses.set(currentIndex, new InetSocketAddress(resolvedAddresses[0], thePort));
    +                        nextAdded = nextRemoved = 1;
    +                        LOG.debug("Newly resolved address: {}", resolvedAddresses[0]);
    +                    } else {
    +                        int i = 0;
    +                        while (i < serverAddresses.size()) {
    +                            if (getHostString(serverAddresses.get(i)).equals(curHostString) &&
    --- End diff --
    
    How to shuffle the elements?


---

[GitHub] zookeeper issue #451: ZOOKEEPER-2184: Zookeeper Client should re-resolve hos...

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

    https://github.com/apache/zookeeper/pull/451
  
    @afine @ijuma I've finished refactoring of StaticHostProvider.
    
    The implementation follows that I explained in my email as **Option-3**:
    
    > - Do not cache IPs, 
    > - Shuffle the names and resolve with getAllByName() every time when next() is called,
    > - Use getAllByName(), but shuffle the list and return the first IP to properly handle hostnames associated with multiple IPs,
    > - JDK's built-in caching will prevent name servers from being flooded and will do the re-resolution automatically when cache expires,
    
    The `Resolver` interface which is also introduced in this patch is the solution for the problem that @afine raised: to properly mock out the `getAllByName()` call in unit tests.


---

[GitHub] zookeeper issue #451: ZOOKEEPER-2184: Zookeeper Client should re-resolve hos...

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

    https://github.com/apache/zookeeper/pull/451
  
    Really happy to see this in a released version, thanks everyone.
    
    @anmolnar after testing this with Apache Kafka, it seems like the combination of a reasonably long backoff (1 second) with randomization when the list is small can cause connection timeouts in some cases that one would not expect. The concrete example we saw was usage of localhost which caused ipv4 and ipv6 addresses to be resolved while the only the ipv4 one worked. We had tests that would fail quite often with a connection timeout of 6 seconds because the ipv6 one would be picked continuously.
    
    @rajinisivaram filed a ZooKeeper issue to track a potential future improvement to this logic:
    https://issues.apache.org/jira/browse/ZOOKEEPER-3100


---

[GitHub] zookeeper pull request #451: ZOOKEEPER-2184: Zookeeper Client should re-reso...

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

    https://github.com/apache/zookeeper/pull/451#discussion_r191034452
  
    --- Diff: src/java/main/org/apache/zookeeper/client/StaticHostProvider.java ---
    @@ -30,76 +31,118 @@
     import org.slf4j.Logger;
     import org.slf4j.LoggerFactory;
     
    -
     /**
      * Most simple HostProvider, resolves only on instantiation.
    --- End diff --
    
    Now we don't only resolve on instantiation, so this comment needs update.


---

[GitHub] zookeeper pull request #451: ZOOKEEPER-2184: Zookeeper Client should re-reso...

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

    https://github.com/apache/zookeeper/pull/451#discussion_r166103404
  
    --- Diff: src/java/main/org/apache/zookeeper/client/StaticHostProvider.java ---
    @@ -58,48 +61,122 @@
         public StaticHostProvider(Collection<InetSocketAddress> serverAddresses) {
             for (InetSocketAddress address : serverAddresses) {
                 try {
    -                InetAddress ia = address.getAddress();
    -                InetAddress resolvedAddresses[] = InetAddress.getAllByName((ia != null) ? ia.getHostAddress() :
    -                        address.getHostName());
    +                InetAddress resolvedAddresses[] = InetAddress.getAllByName(getHostString(address));
                     for (InetAddress resolvedAddress : resolvedAddresses) {
    -                    // If hostName is null but the address is not, we can tell that
    -                    // the hostName is an literal IP address. Then we can set the host string as the hostname
    -                    // safely to avoid reverse DNS lookup.
    -                    // As far as i know, the only way to check if the hostName is null is use toString().
    -                    // Both the two implementations of InetAddress are final class, so we can trust the return value of
    -                    // the toString() method.
    -                    if (resolvedAddress.toString().startsWith("/")
    -                            && resolvedAddress.getAddress() != null) {
    -                        this.serverAddresses.add(
    -                                new InetSocketAddress(InetAddress.getByAddress(
    -                                        address.getHostName(),
    -                                        resolvedAddress.getAddress()),
    -                                        address.getPort()));
    -                    } else {
    -                        this.serverAddresses.add(new InetSocketAddress(resolvedAddress.getHostAddress(), address.getPort()));
    -                    }
    +                    this.serverAddresses.add(new InetSocketAddress(resolvedAddress, address.getPort()));
                     }
                 } catch (UnknownHostException e) {
                     LOG.error("Unable to connect to server: {}", address, e);
                 }
             }
    -        
    +
             if (this.serverAddresses.isEmpty()) {
                 throw new IllegalArgumentException(
                         "A HostProvider may not be empty!");
             }
             Collections.shuffle(this.serverAddresses);
         }
     
    +    /**
    +     * Evaluate to a hostname if one is available and otherwise it returns the
    +     * string representation of the IP address.
    +     *
    +     * In Java 7, we have a method getHostString, but earlier versions do not support it.
    +     * This method is to provide a replacement for InetSocketAddress.getHostString().
    +     *
    +     * @param addr
    +     * @return Hostname string of address parameter
    +     */
    +    private String getHostString(InetSocketAddress addr) {
    +        String hostString = "";
    +
    +        if (addr == null) {
    +            return hostString;
    +        }
    +        if (!addr.isUnresolved()) {
    +            InetAddress ia = addr.getAddress();
    +
    +            // If the string starts with '/', then it has no hostname
    +            // and we want to avoid the reverse lookup, so we return
    +            // the string representation of the address.
    +            if (ia.toString().startsWith("/")) {
    +                hostString = ia.getHostAddress();
    +            } else {
    +                hostString = addr.getHostName();
    +            }
    +        } else {
    +            // According to the Java 6 documentation, if the hostname is
    +            // unresolved, then the string before the colon is the hostname.
    +            String addrString = addr.toString();
    +            hostString = addrString.substring(0, addrString.lastIndexOf(':'));
    +        }
    +
    +        return hostString;
    +    }
    +
         public int size() {
             return serverAddresses.size();
         }
     
    +    // Counts the number of addresses added and removed during
    +    // the last call to next. Used mainly for test purposes.
    +    // See StasticHostProviderTest.
    +    private int nextAdded = 0;
    +    private int nextRemoved = 0;
    +
    +    public int getNextAdded() {
    +        return nextAdded;
    +    }
    +
    +    public int getNextRemoved() {
    +        return nextRemoved;
    +    }
    +
         public InetSocketAddress next(long spinDelay) {
    -        ++currentIndex;
    -        if (currentIndex == serverAddresses.size()) {
    -            currentIndex = 0;
    +        // Handle possible connection error by re-resolving hostname if possible
    +        if (!connectedSinceNext) {
    +            InetSocketAddress curAddr = serverAddresses.get(currentIndex);
    +            String curHostString = getHostString(curAddr);
    +            if (!curHostString.equals(curAddr.getAddress().getHostAddress())) {
    +                LOG.info("Resolving again hostname: {}", getHostString(curAddr));
    +                try {
    +                    int thePort = curAddr.getPort();
    +                    InetAddress resolvedAddresses[] = InetAddress.getAllByName(curHostString);
    +                    nextAdded = 0;
    +                    nextRemoved = 0;
    +                    if (resolvedAddresses.length == 1) {
    +                        serverAddresses.set(currentIndex, new InetSocketAddress(resolvedAddresses[0], thePort));
    +                        nextAdded = nextRemoved = 1;
    +                        LOG.debug("Newly resolved address: {}", resolvedAddresses[0]);
    +                    } else {
    +                        int i = 0;
    +                        while (i < serverAddresses.size()) {
    +                            if (getHostString(serverAddresses.get(i)).equals(curHostString) &&
    --- End diff --
    
    as i mentioned in https://github.com/apache/zookeeper/pull/150/files#r98543324 this all gets a little complicated? What do you think about using a map to trap all these associations?


---

[GitHub] zookeeper issue #451: ZOOKEEPER-2184: Zookeeper Client should re-resolve hos...

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

    https://github.com/apache/zookeeper/pull/451
  
    Should this PR be targeting `branch-3.4` or target `trunk` and then backport to the 3.4 series?


---

[GitHub] zookeeper pull request #451: ZOOKEEPER-2184: Zookeeper Client should re-reso...

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

    https://github.com/apache/zookeeper/pull/451#discussion_r165524377
  
    --- Diff: src/java/main/org/apache/zookeeper/client/StaticHostProvider.java ---
    @@ -25,6 +25,8 @@
     import java.util.Collection;
     import java.util.Collections;
     import java.util.List;
    +import java.lang.reflect.InvocationTargetException;
    --- End diff --
    
    i think these imports are unused, and there are some others elsewhere in the code


---

[GitHub] zookeeper pull request #451: ZOOKEEPER-2184: Zookeeper Client should re-reso...

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

    https://github.com/apache/zookeeper/pull/451#discussion_r166105622
  
    --- Diff: src/java/test/org/apache/zookeeper/test/StaticHostProviderTest.java ---
    @@ -117,8 +116,32 @@ public void testTwoInvalidHostAddresses() {
             list.add(new InetSocketAddress("a", 2181));
             list.add(new InetSocketAddress("b", 2181));
             new StaticHostProvider(list);
    +	}
    +
    +    @Test
    +    public void testReResolving() {
    +        byte size = 1;
    +        ArrayList<InetSocketAddress> list = new ArrayList<InetSocketAddress>(size);
    +
    +        // Test a hostname that resolves to multiple addresses
    +        list.add(InetSocketAddress.createUnresolved("www.apache.org", 1234));
    --- End diff --
    
    I'm wondering if it's possible to mock this out? It would be great if our unit tests were not dependent on some other infrastructure.


---

[GitHub] zookeeper issue #451: ZOOKEEPER-2184: Zookeeper Client should re-resolve hos...

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

    https://github.com/apache/zookeeper/pull/451
  
    Re-resolving at StaticHostProvider level may not be sufficient as InetAddress.getAllByName(String host) itself uses a Java-level cache inside InetAddress and turns to name service (e.g. DNS) only if the host could not be found in the Java-level cache.
    Unfortunately, when Java resolves a new host using the name service, it puts the host and its addresses in the cache with TTL cache FOREVER. 
    This means, once a host gets resolved by Java, it will never again turn to the name service to re-resolve it. If a host's addresses get updated in DNS, the address cache in Java will still contain the old entry forever.
    So re-resolving at StaticHostProvider won't help in this case, as InetAddress.getAllByName(String host) will still return the old address(es) I think.
    Check the getCachedAddresses method inside InetAddress, the get() method of static final class Cache inside InetAddress and sun.net.InetAddressCachePolicy.get() which returns cachePolicy with default value -1 (FOREVER) if it is not overridden by Security properties "networkaddress.cache.ttl" and "networkaddress.cache.negative.ttl".


---

[GitHub] zookeeper pull request #451: ZOOKEEPER-2184: Zookeeper Client should re-reso...

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

    https://github.com/apache/zookeeper/pull/451#discussion_r167013851
  
    --- Diff: src/java/main/org/apache/zookeeper/client/StaticHostProvider.java ---
    @@ -58,48 +61,122 @@
         public StaticHostProvider(Collection<InetSocketAddress> serverAddresses) {
             for (InetSocketAddress address : serverAddresses) {
                 try {
    -                InetAddress ia = address.getAddress();
    -                InetAddress resolvedAddresses[] = InetAddress.getAllByName((ia != null) ? ia.getHostAddress() :
    -                        address.getHostName());
    +                InetAddress resolvedAddresses[] = InetAddress.getAllByName(getHostString(address));
                     for (InetAddress resolvedAddress : resolvedAddresses) {
    -                    // If hostName is null but the address is not, we can tell that
    -                    // the hostName is an literal IP address. Then we can set the host string as the hostname
    -                    // safely to avoid reverse DNS lookup.
    -                    // As far as i know, the only way to check if the hostName is null is use toString().
    -                    // Both the two implementations of InetAddress are final class, so we can trust the return value of
    -                    // the toString() method.
    -                    if (resolvedAddress.toString().startsWith("/")
    -                            && resolvedAddress.getAddress() != null) {
    -                        this.serverAddresses.add(
    -                                new InetSocketAddress(InetAddress.getByAddress(
    -                                        address.getHostName(),
    -                                        resolvedAddress.getAddress()),
    -                                        address.getPort()));
    -                    } else {
    -                        this.serverAddresses.add(new InetSocketAddress(resolvedAddress.getHostAddress(), address.getPort()));
    -                    }
    +                    this.serverAddresses.add(new InetSocketAddress(resolvedAddress, address.getPort()));
                     }
                 } catch (UnknownHostException e) {
                     LOG.error("Unable to connect to server: {}", address, e);
                 }
             }
    -        
    +
             if (this.serverAddresses.isEmpty()) {
                 throw new IllegalArgumentException(
                         "A HostProvider may not be empty!");
             }
             Collections.shuffle(this.serverAddresses);
         }
     
    +    /**
    +     * Evaluate to a hostname if one is available and otherwise it returns the
    +     * string representation of the IP address.
    +     *
    +     * In Java 7, we have a method getHostString, but earlier versions do not support it.
    +     * This method is to provide a replacement for InetSocketAddress.getHostString().
    +     *
    +     * @param addr
    +     * @return Hostname string of address parameter
    +     */
    +    private String getHostString(InetSocketAddress addr) {
    +        String hostString = "";
    +
    +        if (addr == null) {
    +            return hostString;
    +        }
    +        if (!addr.isUnresolved()) {
    +            InetAddress ia = addr.getAddress();
    +
    +            // If the string starts with '/', then it has no hostname
    +            // and we want to avoid the reverse lookup, so we return
    +            // the string representation of the address.
    +            if (ia.toString().startsWith("/")) {
    +                hostString = ia.getHostAddress();
    +            } else {
    +                hostString = addr.getHostName();
    +            }
    +        } else {
    +            // According to the Java 6 documentation, if the hostname is
    +            // unresolved, then the string before the colon is the hostname.
    +            String addrString = addr.toString();
    +            hostString = addrString.substring(0, addrString.lastIndexOf(':'));
    +        }
    +
    +        return hostString;
    +    }
    +
         public int size() {
             return serverAddresses.size();
         }
     
    +    // Counts the number of addresses added and removed during
    +    // the last call to next. Used mainly for test purposes.
    +    // See StasticHostProviderTest.
    +    private int nextAdded = 0;
    +    private int nextRemoved = 0;
    +
    +    public int getNextAdded() {
    +        return nextAdded;
    +    }
    +
    +    public int getNextRemoved() {
    +        return nextRemoved;
    +    }
    +
         public InetSocketAddress next(long spinDelay) {
    -        ++currentIndex;
    -        if (currentIndex == serverAddresses.size()) {
    -            currentIndex = 0;
    +        // Handle possible connection error by re-resolving hostname if possible
    +        if (!connectedSinceNext) {
    +            InetSocketAddress curAddr = serverAddresses.get(currentIndex);
    +            String curHostString = getHostString(curAddr);
    +            if (!curHostString.equals(curAddr.getAddress().getHostAddress())) {
    +                LOG.info("Resolving again hostname: {}", getHostString(curAddr));
    +                try {
    +                    int thePort = curAddr.getPort();
    +                    InetAddress resolvedAddresses[] = InetAddress.getAllByName(curHostString);
    +                    nextAdded = 0;
    +                    nextRemoved = 0;
    +                    if (resolvedAddresses.length == 1) {
    +                        serverAddresses.set(currentIndex, new InetSocketAddress(resolvedAddresses[0], thePort));
    +                        nextAdded = nextRemoved = 1;
    +                        LOG.debug("Newly resolved address: {}", resolvedAddresses[0]);
    +                    } else {
    +                        int i = 0;
    +                        while (i < serverAddresses.size()) {
    +                            if (getHostString(serverAddresses.get(i)).equals(curHostString) &&
    --- End diff --
    
    You could put the `keySet` in a list and shuffle?


---

[GitHub] zookeeper issue #451: ZOOKEEPER-2184: Zookeeper Client should re-resolve hos...

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

    https://github.com/apache/zookeeper/pull/451
  
    @anmolnar, it's been more than 1 month since the last comment on this PR. Is there anything that still needs to be addressed? The original PR was submitted in January 2017 and it got stalled after a while, I'm hoping the same doesn't happen here. :)


---

[GitHub] zookeeper pull request #451: ZOOKEEPER-2184: Zookeeper Client should re-reso...

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

    https://github.com/apache/zookeeper/pull/451#discussion_r168572502
  
    --- Diff: src/java/test/org/apache/zookeeper/test/ClientPortBindTest.java ---
    @@ -104,7 +104,7 @@ public void testBindByAddress() throws Exception {
             try {
                 startSignal.await(CONNECTION_TIMEOUT,
                         TimeUnit.MILLISECONDS);
    -            Assert.assertTrue("count == 0", startSignal.getCount() == 0);
    +            Assert.assertTrue("count == " + startSignal.getCount(), startSignal.getCount() == 0);
    --- End diff --
    
    I'm not a huge fan of calling `getCount` more than once, since I think the value could change between invocations. Why not just use the return value from `await` in the line above? 


---

[GitHub] zookeeper issue #451: ZOOKEEPER-2184: Zookeeper Client should re-resolve hos...

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

    https://github.com/apache/zookeeper/pull/451
  
    @mfenes The only solution I can think of is to set DNS cache TTL `networkaddress.cache.ttl` to a configurable, non-infinite value.


---

[GitHub] zookeeper pull request #451: ZOOKEEPER-2184: Zookeeper Client should re-reso...

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

    https://github.com/apache/zookeeper/pull/451#discussion_r190899356
  
    --- Diff: src/java/main/org/apache/zookeeper/client/HostProvider.java ---
    @@ -53,7 +54,7 @@
          * @param spinDelay
          *            Milliseconds to wait if all hosts have been tried once.
          */
    -    public InetSocketAddress next(long spinDelay);
    +    public InetSocketAddress next(long spinDelay) throws UnknownHostException;
    --- End diff --
    
    @hanm This is done.


---

[GitHub] zookeeper issue #451: ZOOKEEPER-2184: Zookeeper Client should re-resolve hos...

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

    https://github.com/apache/zookeeper/pull/451
  
    @fpj I think this patch is ready for merging as it is. Are you still having concerns?


---

[GitHub] zookeeper pull request #451: ZOOKEEPER-2184: Zookeeper Client should re-reso...

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

    https://github.com/apache/zookeeper/pull/451#discussion_r186741712
  
    --- Diff: src/java/main/org/apache/zookeeper/client/StaticHostProvider.java ---
    @@ -30,76 +31,118 @@
     import org.slf4j.Logger;
     import org.slf4j.LoggerFactory;
     
    -
     /**
      * Most simple HostProvider, resolves only on instantiation.
    - * 
    + *
      */
     @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);
     
    -    private final List<InetSocketAddress> serverAddresses = new ArrayList<InetSocketAddress>(
    -            5);
    +    private final List<InetSocketAddress> serverAddresses = new ArrayList<InetSocketAddress>(5);
     
         private int lastIndex = -1;
     
         private int currentIndex = -1;
     
    +    private Resolver resolver;
    +
         /**
          * Constructs a SimpleHostSet.
    -     * 
    +     *
          * @param serverAddresses
          *            possibly unresolved ZooKeeper server addresses
          * @throws IllegalArgumentException
          *             if serverAddresses is empty or resolves to an empty list
          */
         public StaticHostProvider(Collection<InetSocketAddress> serverAddresses) {
    -        for (InetSocketAddress address : serverAddresses) {
    -            try {
    -                InetAddress ia = address.getAddress();
    -                InetAddress resolvedAddresses[] = InetAddress.getAllByName((ia != null) ? ia.getHostAddress() :
    -                        address.getHostName());
    -                for (InetAddress resolvedAddress : resolvedAddresses) {
    -                    // If hostName is null but the address is not, we can tell that
    -                    // the hostName is an literal IP address. Then we can set the host string as the hostname
    -                    // safely to avoid reverse DNS lookup.
    -                    // As far as i know, the only way to check if the hostName is null is use toString().
    -                    // Both the two implementations of InetAddress are final class, so we can trust the return value of
    -                    // the toString() method.
    -                    if (resolvedAddress.toString().startsWith("/")
    -                            && resolvedAddress.getAddress() != null) {
    -                        this.serverAddresses.add(
    -                                new InetSocketAddress(InetAddress.getByAddress(
    -                                        address.getHostName(),
    -                                        resolvedAddress.getAddress()),
    -                                        address.getPort()));
    -                    } else {
    -                        this.serverAddresses.add(new InetSocketAddress(resolvedAddress.getHostAddress(), address.getPort()));
    -                    }
    -                }
    -            } catch (UnknownHostException e) {
    -                LOG.error("Unable to connect to server: {}", address, e);
    +        this.resolver = new Resolver() {
    +            @Override
    +            public InetAddress[] getAllByName(String name) throws UnknownHostException {
    +                return InetAddress.getAllByName(name);
                 }
    -        }
    -        
    -        if (this.serverAddresses.isEmpty()) {
    +        };
    +        init(serverAddresses);
    +    }
    +
    +    /**
    +     * 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
    +     * @throws IllegalArgumentException
    +     *             if serverAddresses is empty or resolves to an empty list
    +     */
    +    public StaticHostProvider(Collection<InetSocketAddress> serverAddresses, Resolver resolver) {
    +        this.resolver = resolver;
    +        init(serverAddresses);
    +    }
    +
    +    /**
    +     * Common init method for all constructors.
    +     * Resolve all unresolved server addresses, put them in a list and shuffle.
    +     */
    +    private void init(Collection<InetSocketAddress> serverAddresses) {
    +        if (serverAddresses.isEmpty()) {
                 throw new IllegalArgumentException(
                         "A HostProvider may not be empty!");
             }
    +
    +        this.serverAddresses.addAll(serverAddresses);
             Collections.shuffle(this.serverAddresses);
         }
     
    +    /**
    +     * Evaluate to a hostname if one is available and otherwise it returns the
    +     * string representation of the IP address.
    +     *
    +     * In Java 7, we have a method getHostString, but earlier versions do not support it.
    +     * This method is to provide a replacement for InetSocketAddress.getHostString().
    +     *
    +     * @param addr
    +     * @return Hostname string of address parameter
    +     */
    +    private String getHostString(InetSocketAddress addr) {
    --- End diff --
    
    Afaik 3.4 should compile with JDK6


---

[GitHub] zookeeper pull request #451: ZOOKEEPER-2184: Zookeeper Client should re-reso...

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

    https://github.com/apache/zookeeper/pull/451#discussion_r186690327
  
    --- Diff: src/java/main/org/apache/zookeeper/client/StaticHostProvider.java ---
    @@ -111,9 +154,18 @@ public InetSocketAddress next(long spinDelay) {
                 lastIndex = 0;
             }
     
    -        return serverAddresses.get(currentIndex);
    +        InetSocketAddress curAddr = serverAddresses.get(currentIndex);
    +
    +        String curHostString = getHostString(curAddr);
    +        List<InetAddress> resolvedAddresses = new ArrayList<InetAddress>(Arrays.asList(this.resolver.getAllByName(curHostString)));
    +        if (resolvedAddresses.isEmpty()) {
    +            throw new UnknownHostException("No IP address returned for address: " + curHostString);
    +        }
    +        Collections.shuffle(resolvedAddresses);
    --- End diff --
    
    Ok.


---

[GitHub] zookeeper issue #451: ZOOKEEPER-2184: Zookeeper Client should re-resolve hos...

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

    https://github.com/apache/zookeeper/pull/451
  
    @hanm I updated the pull request according to your suggestions. Please take a look.
    I also updated javadoc accordingly. ZooKeeper docs could be updated in a separate PR as suggested.


---

[GitHub] zookeeper pull request #451: ZOOKEEPER-2184: Zookeeper Client should re-reso...

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

    https://github.com/apache/zookeeper/pull/451#discussion_r186690242
  
    --- Diff: src/java/main/org/apache/zookeeper/client/StaticHostProvider.java ---
    @@ -111,9 +154,18 @@ public InetSocketAddress next(long spinDelay) {
                 lastIndex = 0;
             }
     
    -        return serverAddresses.get(currentIndex);
    +        InetSocketAddress curAddr = serverAddresses.get(currentIndex);
    +
    +        String curHostString = getHostString(curAddr);
    +        List<InetAddress> resolvedAddresses = new ArrayList<InetAddress>(Arrays.asList(this.resolver.getAllByName(curHostString)));
    +        if (resolvedAddresses.isEmpty()) {
    +            throw new UnknownHostException("No IP address returned for address: " + curHostString);
    --- End diff --
    
    I think it is fine to loop indefinitely because the client needs it for progress. We need a way to break the loop in the case the client closes, though.


---

[GitHub] zookeeper pull request #451: ZOOKEEPER-2184: Zookeeper Client should re-reso...

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

    https://github.com/apache/zookeeper/pull/451#discussion_r168578617
  
    --- Diff: src/java/main/org/apache/zookeeper/client/StaticHostProvider.java ---
    @@ -58,48 +61,122 @@
         public StaticHostProvider(Collection<InetSocketAddress> serverAddresses) {
             for (InetSocketAddress address : serverAddresses) {
                 try {
    -                InetAddress ia = address.getAddress();
    -                InetAddress resolvedAddresses[] = InetAddress.getAllByName((ia != null) ? ia.getHostAddress() :
    -                        address.getHostName());
    +                InetAddress resolvedAddresses[] = InetAddress.getAllByName(getHostString(address));
                     for (InetAddress resolvedAddress : resolvedAddresses) {
    -                    // If hostName is null but the address is not, we can tell that
    -                    // the hostName is an literal IP address. Then we can set the host string as the hostname
    -                    // safely to avoid reverse DNS lookup.
    -                    // As far as i know, the only way to check if the hostName is null is use toString().
    -                    // Both the two implementations of InetAddress are final class, so we can trust the return value of
    -                    // the toString() method.
    -                    if (resolvedAddress.toString().startsWith("/")
    -                            && resolvedAddress.getAddress() != null) {
    -                        this.serverAddresses.add(
    -                                new InetSocketAddress(InetAddress.getByAddress(
    -                                        address.getHostName(),
    -                                        resolvedAddress.getAddress()),
    -                                        address.getPort()));
    -                    } else {
    -                        this.serverAddresses.add(new InetSocketAddress(resolvedAddress.getHostAddress(), address.getPort()));
    -                    }
    +                    this.serverAddresses.add(new InetSocketAddress(resolvedAddress, address.getPort()));
                     }
                 } catch (UnknownHostException e) {
                     LOG.error("Unable to connect to server: {}", address, e);
                 }
             }
    -        
    +
             if (this.serverAddresses.isEmpty()) {
                 throw new IllegalArgumentException(
                         "A HostProvider may not be empty!");
             }
             Collections.shuffle(this.serverAddresses);
         }
     
    +    /**
    +     * Evaluate to a hostname if one is available and otherwise it returns the
    +     * string representation of the IP address.
    +     *
    +     * In Java 7, we have a method getHostString, but earlier versions do not support it.
    +     * This method is to provide a replacement for InetSocketAddress.getHostString().
    +     *
    +     * @param addr
    +     * @return Hostname string of address parameter
    +     */
    +    private String getHostString(InetSocketAddress addr) {
    +        String hostString = "";
    +
    +        if (addr == null) {
    +            return hostString;
    +        }
    +        if (!addr.isUnresolved()) {
    +            InetAddress ia = addr.getAddress();
    +
    +            // If the string starts with '/', then it has no hostname
    +            // and we want to avoid the reverse lookup, so we return
    +            // the string representation of the address.
    +            if (ia.toString().startsWith("/")) {
    +                hostString = ia.getHostAddress();
    +            } else {
    +                hostString = addr.getHostName();
    +            }
    +        } else {
    +            // According to the Java 6 documentation, if the hostname is
    +            // unresolved, then the string before the colon is the hostname.
    +            String addrString = addr.toString();
    +            hostString = addrString.substring(0, addrString.lastIndexOf(':'));
    +        }
    +
    +        return hostString;
    +    }
    +
         public int size() {
             return serverAddresses.size();
         }
     
    +    // Counts the number of addresses added and removed during
    +    // the last call to next. Used mainly for test purposes.
    +    // See StasticHostProviderTest.
    +    private int nextAdded = 0;
    +    private int nextRemoved = 0;
    +
    +    public int getNextAdded() {
    +        return nextAdded;
    +    }
    +
    +    public int getNextRemoved() {
    +        return nextRemoved;
    +    }
    +
         public InetSocketAddress next(long spinDelay) {
    -        ++currentIndex;
    -        if (currentIndex == serverAddresses.size()) {
    -            currentIndex = 0;
    +        // Handle possible connection error by re-resolving hostname if possible
    +        if (!connectedSinceNext) {
    +            InetSocketAddress curAddr = serverAddresses.get(currentIndex);
    +            String curHostString = getHostString(curAddr);
    +            if (!curHostString.equals(curAddr.getAddress().getHostAddress())) {
    +                LOG.info("Resolving again hostname: {}", getHostString(curAddr));
    +                try {
    +                    int thePort = curAddr.getPort();
    +                    InetAddress resolvedAddresses[] = InetAddress.getAllByName(curHostString);
    +                    nextAdded = 0;
    +                    nextRemoved = 0;
    +                    if (resolvedAddresses.length == 1) {
    +                        serverAddresses.set(currentIndex, new InetSocketAddress(resolvedAddresses[0], thePort));
    +                        nextAdded = nextRemoved = 1;
    +                        LOG.debug("Newly resolved address: {}", resolvedAddresses[0]);
    +                    } else {
    +                        int i = 0;
    +                        while (i < serverAddresses.size()) {
    +                            if (getHostString(serverAddresses.get(i)).equals(curHostString) &&
    --- End diff --
    
    Why don't we simplify things and just do this always? In other words, always remove all serverAddresses that correspond to the curHostString. That way if curHostString goes from resolving to many addresses to just 1, we remove all of them?


---

[GitHub] zookeeper pull request #451: ZOOKEEPER-2184: Zookeeper Client should re-reso...

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

    https://github.com/apache/zookeeper/pull/451#discussion_r191252989
  
    --- Diff: src/java/test/org/apache/zookeeper/test/ClientPortBindTest.java ---
    @@ -102,9 +102,9 @@ public void testBindByAddress() throws Exception {
             startSignal = new CountDownLatch(1);
             ZooKeeper zk = new ZooKeeper(HOSTPORT, CONNECTION_TIMEOUT, this);
             try {
    -            startSignal.await(CONNECTION_TIMEOUT,
    --- End diff --
    
    Sure, I agree. I think that was a suggestion from @afine and it's completely reasonable.


---

[GitHub] zookeeper pull request #451: ZOOKEEPER-2184: Zookeeper Client should re-reso...

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

    https://github.com/apache/zookeeper/pull/451#discussion_r166479687
  
    --- Diff: src/java/test/org/apache/zookeeper/test/StaticHostProviderTest.java ---
    @@ -117,8 +116,32 @@ public void testTwoInvalidHostAddresses() {
             list.add(new InetSocketAddress("a", 2181));
             list.add(new InetSocketAddress("b", 2181));
             new StaticHostProvider(list);
    +	}
    +
    +    @Test
    +    public void testReResolving() {
    +        byte size = 1;
    +        ArrayList<InetSocketAddress> list = new ArrayList<InetSocketAddress>(size);
    +
    +        // Test a hostname that resolves to multiple addresses
    +        list.add(InetSocketAddress.createUnresolved("www.apache.org", 1234));
    --- End diff --
    
    Yeah, this is annoying. Although, another possibility would be to put the dns calls in a method and then subclass `StaticHostProvider` for the tests and overriding this method to return what you want. 


---

[GitHub] zookeeper pull request #451: ZOOKEEPER-2184: Zookeeper Client should re-reso...

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

    https://github.com/apache/zookeeper/pull/451#discussion_r168524336
  
    --- Diff: src/java/test/org/apache/zookeeper/test/StaticHostProviderTest.java ---
    @@ -117,8 +116,32 @@ public void testTwoInvalidHostAddresses() {
             list.add(new InetSocketAddress("a", 2181));
             list.add(new InetSocketAddress("b", 2181));
             new StaticHostProvider(list);
    +	}
    +
    +    @Test
    +    public void testReResolving() {
    +        byte size = 1;
    +        ArrayList<InetSocketAddress> list = new ArrayList<InetSocketAddress>(size);
    +
    +        // Test a hostname that resolves to multiple addresses
    +        list.add(InetSocketAddress.createUnresolved("www.apache.org", 1234));
    --- End diff --
    
    This is done.


---

[GitHub] zookeeper pull request #451: ZOOKEEPER-2184: Zookeeper Client should re-reso...

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

    https://github.com/apache/zookeeper/pull/451#discussion_r166329760
  
    --- Diff: src/java/test/org/apache/zookeeper/test/StaticHostProviderTest.java ---
    @@ -117,8 +116,32 @@ public void testTwoInvalidHostAddresses() {
             list.add(new InetSocketAddress("a", 2181));
             list.add(new InetSocketAddress("b", 2181));
             new StaticHostProvider(list);
    +	}
    +
    +    @Test
    +    public void testReResolving() {
    +        byte size = 1;
    +        ArrayList<InetSocketAddress> list = new ArrayList<InetSocketAddress>(size);
    +
    +        // Test a hostname that resolves to multiple addresses
    +        list.add(InetSocketAddress.createUnresolved("www.apache.org", 1234));
    --- End diff --
    
    Good point I spent hours finding a way for it. Unfortunately static members can't be mocked with Mockito. We should use PowerMock for it (as described here https://blog.codecentric.de/en/2016/03/junit-testing-using-mockito-powermock/), but that'd be a new dependency not sure if it's acceptable here.


---

[GitHub] zookeeper issue #451: ZOOKEEPER-2184: Zookeeper Client should re-resolve hos...

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

    https://github.com/apache/zookeeper/pull/451
  
    I agree with @ijuma 
    @afine You were involved in this patch too. Are you willing to merge it in @fpj 's absence?


---

[GitHub] zookeeper pull request #451: ZOOKEEPER-2184: Zookeeper Client should re-reso...

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

    https://github.com/apache/zookeeper/pull/451#discussion_r166286530
  
    --- Diff: src/java/test/org/apache/zookeeper/test/ReadOnlyModeTest.java ---
    @@ -239,13 +243,13 @@ public void testSessionEstablishment() throws Exception {
         public void testSeekForRwServer() throws Exception {
     
             // setup the logger to capture all logs
    -        Layout layout = Logger.getRootLogger().getAppender("CONSOLE")
    +        Layout layout = org.apache.log4j.Logger.getRootLogger().getAppender("CONSOLE")
    --- End diff --
    
    I few lines above there're org.apache.log4j references which conflicts with the 'Logger' class, hence it needs to be explicitly referenced.


---

[GitHub] zookeeper issue #451: ZOOKEEPER-2184: Zookeeper Client should re-resolve hos...

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

    https://github.com/apache/zookeeper/pull/451
  
    @jeffwidman 
    
    > Should this PR be targeting branch-3.4 or target trunk and then backport to the 3.4 series?
    
    The original PR targets 3.4 which is explained in a comment from @fpj on the jira:
    https://issues.apache.org/jira/browse/ZOOKEEPER-2184?focusedCommentId=15823099&page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel#comment-15823099
    
    > For the 3.5 branch, we will need a different patch because of the reconfiguration changes to StaticHostProvider. I'll work on it once the 3.4 patch gets a +1.



---

[GitHub] zookeeper pull request #451: ZOOKEEPER-2184: Zookeeper Client should re-reso...

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

    https://github.com/apache/zookeeper/pull/451#discussion_r168764368
  
    --- Diff: src/java/test/org/apache/zookeeper/test/StaticHostProviderTest.java ---
    @@ -119,6 +120,68 @@ public void testTwoInvalidHostAddresses() {
             new StaticHostProvider(list);
         }
     
    +    @Test
    +    public void testReResolvingSingle() {
    +        byte size = 1;
    +        ArrayList<InetSocketAddress> list = new ArrayList<InetSocketAddress>(size);
    +
    +        // Test a hostname that resolves to a single address
    +        list.clear();
    --- End diff --
    
    Removed.


---

[GitHub] zookeeper pull request #451: ZOOKEEPER-2184: Zookeeper Client should re-reso...

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

    https://github.com/apache/zookeeper/pull/451#discussion_r166669488
  
    --- Diff: src/java/test/org/apache/zookeeper/test/ReadOnlyModeTest.java ---
    @@ -239,13 +243,13 @@ public void testSessionEstablishment() throws Exception {
         public void testSeekForRwServer() throws Exception {
     
             // setup the logger to capture all logs
    -        Layout layout = Logger.getRootLogger().getAppender("CONSOLE")
    +        Layout layout = org.apache.log4j.Logger.getRootLogger().getAppender("CONSOLE")
    --- End diff --
    
    slf4j's string formatter feature is being used in these tests. I wouldn't refactor them to be honest.


---

[GitHub] zookeeper pull request #451: ZOOKEEPER-2184: Zookeeper Client should re-reso...

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

    https://github.com/apache/zookeeper/pull/451#discussion_r166671671
  
    --- Diff: src/java/main/org/apache/zookeeper/client/StaticHostProvider.java ---
    @@ -58,48 +61,122 @@
         public StaticHostProvider(Collection<InetSocketAddress> serverAddresses) {
             for (InetSocketAddress address : serverAddresses) {
                 try {
    -                InetAddress ia = address.getAddress();
    -                InetAddress resolvedAddresses[] = InetAddress.getAllByName((ia != null) ? ia.getHostAddress() :
    -                        address.getHostName());
    +                InetAddress resolvedAddresses[] = InetAddress.getAllByName(getHostString(address));
                     for (InetAddress resolvedAddress : resolvedAddresses) {
    -                    // If hostName is null but the address is not, we can tell that
    -                    // the hostName is an literal IP address. Then we can set the host string as the hostname
    -                    // safely to avoid reverse DNS lookup.
    -                    // As far as i know, the only way to check if the hostName is null is use toString().
    -                    // Both the two implementations of InetAddress are final class, so we can trust the return value of
    -                    // the toString() method.
    -                    if (resolvedAddress.toString().startsWith("/")
    -                            && resolvedAddress.getAddress() != null) {
    -                        this.serverAddresses.add(
    -                                new InetSocketAddress(InetAddress.getByAddress(
    -                                        address.getHostName(),
    -                                        resolvedAddress.getAddress()),
    -                                        address.getPort()));
    -                    } else {
    -                        this.serverAddresses.add(new InetSocketAddress(resolvedAddress.getHostAddress(), address.getPort()));
    -                    }
    +                    this.serverAddresses.add(new InetSocketAddress(resolvedAddress, address.getPort()));
                     }
                 } catch (UnknownHostException e) {
                     LOG.error("Unable to connect to server: {}", address, e);
                 }
             }
    -        
    +
             if (this.serverAddresses.isEmpty()) {
                 throw new IllegalArgumentException(
                         "A HostProvider may not be empty!");
             }
             Collections.shuffle(this.serverAddresses);
         }
     
    +    /**
    +     * Evaluate to a hostname if one is available and otherwise it returns the
    +     * string representation of the IP address.
    +     *
    +     * In Java 7, we have a method getHostString, but earlier versions do not support it.
    +     * This method is to provide a replacement for InetSocketAddress.getHostString().
    +     *
    +     * @param addr
    +     * @return Hostname string of address parameter
    +     */
    +    private String getHostString(InetSocketAddress addr) {
    +        String hostString = "";
    +
    +        if (addr == null) {
    +            return hostString;
    +        }
    +        if (!addr.isUnresolved()) {
    +            InetAddress ia = addr.getAddress();
    +
    +            // If the string starts with '/', then it has no hostname
    +            // and we want to avoid the reverse lookup, so we return
    +            // the string representation of the address.
    +            if (ia.toString().startsWith("/")) {
    +                hostString = ia.getHostAddress();
    +            } else {
    +                hostString = addr.getHostName();
    +            }
    +        } else {
    +            // According to the Java 6 documentation, if the hostname is
    +            // unresolved, then the string before the colon is the hostname.
    +            String addrString = addr.toString();
    +            hostString = addrString.substring(0, addrString.lastIndexOf(':'));
    +        }
    +
    +        return hostString;
    +    }
    +
         public int size() {
             return serverAddresses.size();
         }
     
    +    // Counts the number of addresses added and removed during
    +    // the last call to next. Used mainly for test purposes.
    +    // See StasticHostProviderTest.
    +    private int nextAdded = 0;
    +    private int nextRemoved = 0;
    +
    +    public int getNextAdded() {
    +        return nextAdded;
    +    }
    +
    +    public int getNextRemoved() {
    +        return nextRemoved;
    +    }
    +
         public InetSocketAddress next(long spinDelay) {
    -        ++currentIndex;
    -        if (currentIndex == serverAddresses.size()) {
    -            currentIndex = 0;
    +        // Handle possible connection error by re-resolving hostname if possible
    +        if (!connectedSinceNext) {
    +            InetSocketAddress curAddr = serverAddresses.get(currentIndex);
    +            String curHostString = getHostString(curAddr);
    +            if (!curHostString.equals(curAddr.getAddress().getHostAddress())) {
    +                LOG.info("Resolving again hostname: {}", getHostString(curAddr));
    +                try {
    +                    int thePort = curAddr.getPort();
    +                    InetAddress resolvedAddresses[] = InetAddress.getAllByName(curHostString);
    +                    nextAdded = 0;
    +                    nextRemoved = 0;
    +                    if (resolvedAddresses.length == 1) {
    --- End diff --
    
    It might be easier to just remove the special case of `resolvedAddresses.length == 1` and let the other part iterate over the list and remove the affected elements.


---

[GitHub] zookeeper pull request #451: ZOOKEEPER-2184: Zookeeper Client should re-reso...

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

    https://github.com/apache/zookeeper/pull/451#discussion_r186553219
  
    --- Diff: src/java/main/org/apache/zookeeper/client/StaticHostProvider.java ---
    @@ -111,9 +154,18 @@ public InetSocketAddress next(long spinDelay) {
                 lastIndex = 0;
             }
     
    -        return serverAddresses.get(currentIndex);
    +        InetSocketAddress curAddr = serverAddresses.get(currentIndex);
    +
    +        String curHostString = getHostString(curAddr);
    +        List<InetAddress> resolvedAddresses = new ArrayList<InetAddress>(Arrays.asList(this.resolver.getAllByName(curHostString)));
    +        if (resolvedAddresses.isEmpty()) {
    +            throw new UnknownHostException("No IP address returned for address: " + curHostString);
    +        }
    +        Collections.shuffle(resolvedAddresses);
    --- End diff --
    
    It doesn't't hurt, but do we really need to shuffle the resolved addresses?


---

[GitHub] zookeeper issue #451: ZOOKEEPER-2184: Zookeeper Client should re-resolve hos...

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

    https://github.com/apache/zookeeper/pull/451
  
    Just confirmed on 3.4 branch: ZK uses 30 secs cache TTL on my mac.


---

[GitHub] zookeeper pull request #451: ZOOKEEPER-2184: Zookeeper Client should re-reso...

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

    https://github.com/apache/zookeeper/pull/451#discussion_r166286605
  
    --- Diff: src/java/main/org/apache/zookeeper/client/StaticHostProvider.java ---
    @@ -6,9 +6,9 @@
      * to you under the Apache License, Version 2.0 (the
      * "License"); you may not use this file except in compliance
      * with the License.  You may obtain a copy of the License at
    - *
    - *     http://www.apache.org/licenses/LICENSE-2.0
    - *
    + * <p>
    --- End diff --
    
    Yes it was.


---

[GitHub] zookeeper issue #451: ZOOKEEPER-2184: Zookeeper Client should re-resolve hos...

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

    https://github.com/apache/zookeeper/pull/451
  
    Going one step back I wonder why we try to deal with multiple addresses at all.
    
    HostProvider should just make a transformation from unresolved InetSocketAddresses to resolved InetSocketAddresses. The easiest way as I can see it is to create a new instance of InetSocketAddress if the input is unresolved every time `next()` is called. Otherwise just pass it through.
    
    JVM will deal with the rest: resolution, caching and re-resolution once the cache is expires (30 secs).


---

[GitHub] zookeeper issue #451: ZOOKEEPER-2184: Zookeeper Client should re-resolve hos...

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

    https://github.com/apache/zookeeper/pull/451
  
    Thanks @hanm 


---

[GitHub] zookeeper pull request #451: ZOOKEEPER-2184: Zookeeper Client should re-reso...

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

    https://github.com/apache/zookeeper/pull/451#discussion_r186591140
  
    --- Diff: src/java/main/org/apache/zookeeper/client/HostProvider.java ---
    @@ -53,7 +54,7 @@
          * @param spinDelay
          *            Milliseconds to wait if all hosts have been tried once.
          */
    -    public InetSocketAddress next(long spinDelay);
    +    public InetSocketAddress next(long spinDelay) throws UnknownHostException;
    --- End diff --
    
    That's a very good point actually, can't remember why implemented this way. 
    
    I checked the callers' side and both fall into the same try-catch report loop which reports the problem and retry calling `next()` method. I think the reason was to keep the logic simple here and don't implement something which is already handled by someone else.
    
    But because it's a change on a public API, it would be probably better to do the retry logic here.


---

[GitHub] zookeeper pull request #451: ZOOKEEPER-2184: Zookeeper Client should re-reso...

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

    https://github.com/apache/zookeeper/pull/451#discussion_r168576090
  
    --- Diff: src/java/main/org/apache/zookeeper/client/StaticHostProvider.java ---
    @@ -47,59 +51,169 @@
     
         private int currentIndex = -1;
     
    +    // Don't re-resolve on first next() call
    +    private boolean connectedSinceNext = true;
    +
    +    private Resolver resolver;
    +
         /**
          * Constructs a SimpleHostSet.
    -     * 
    +     *
          * @param serverAddresses
          *            possibly unresolved ZooKeeper server addresses
          * @throws IllegalArgumentException
          *             if serverAddresses is empty or resolves to an empty list
          */
         public StaticHostProvider(Collection<InetSocketAddress> serverAddresses) {
    +        this.resolver = new Resolver() {
    +            @Override
    +            public InetAddress[] getAllByName(String name) throws UnknownHostException {
    +                return InetAddress.getAllByName(name);
    +            }
    +        };
    +        init(serverAddresses);
    +    }
    +
    +    /**
    +     * 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
    +     * @throws IllegalArgumentException
    +     *             if serverAddresses is empty or resolves to an empty list
    +     */
    +    public StaticHostProvider(Collection<InetSocketAddress> serverAddresses, Resolver resolver) {
    +        this.resolver = resolver;
    +        init(serverAddresses);
    +    }
    +
    +    /**
    +     * Common init method for all constructors.
    +     * Resolve all unresolved server addresses, put them in a list and shuffle.
    +     */
    +    private void init(Collection<InetSocketAddress> serverAddresses) {
             for (InetSocketAddress address : serverAddresses) {
                 try {
    -                InetAddress ia = address.getAddress();
    -                InetAddress resolvedAddresses[] = InetAddress.getAllByName((ia != null) ? ia.getHostAddress() :
    -                        address.getHostName());
    +                InetAddress resolvedAddresses[] = this.resolver.getAllByName(getHostString(address));
                     for (InetAddress resolvedAddress : resolvedAddresses) {
    -                    // If hostName is null but the address is not, we can tell that
    -                    // the hostName is an literal IP address. Then we can set the host string as the hostname
    -                    // safely to avoid reverse DNS lookup.
    -                    // As far as i know, the only way to check if the hostName is null is use toString().
    -                    // Both the two implementations of InetAddress are final class, so we can trust the return value of
    -                    // the toString() method.
    -                    if (resolvedAddress.toString().startsWith("/")
    -                            && resolvedAddress.getAddress() != null) {
    -                        this.serverAddresses.add(
    -                                new InetSocketAddress(InetAddress.getByAddress(
    -                                        address.getHostName(),
    -                                        resolvedAddress.getAddress()),
    -                                        address.getPort()));
    -                    } else {
    -                        this.serverAddresses.add(new InetSocketAddress(resolvedAddress.getHostAddress(), address.getPort()));
    -                    }
    +                    this.serverAddresses.add(new InetSocketAddress(resolvedAddress, address.getPort()));
                     }
                 } catch (UnknownHostException e) {
                     LOG.error("Unable to connect to server: {}", address, e);
                 }
             }
    -        
    +
             if (this.serverAddresses.isEmpty()) {
                 throw new IllegalArgumentException(
                         "A HostProvider may not be empty!");
             }
    +
             Collections.shuffle(this.serverAddresses);
         }
     
    +    /**
    +     * Evaluate to a hostname if one is available and otherwise it returns the
    +     * string representation of the IP address.
    +     *
    +     * In Java 7, we have a method getHostString, but earlier versions do not support it.
    +     * This method is to provide a replacement for InetSocketAddress.getHostString().
    +     *
    +     * @param addr
    +     * @return Hostname string of address parameter
    +     */
    +    private String getHostString(InetSocketAddress addr) {
    +        String hostString = "";
    +
    +        if (addr == null) {
    +            return hostString;
    +        }
    +        if (!addr.isUnresolved()) {
    +            InetAddress ia = addr.getAddress();
    +
    +            // If the string starts with '/', then it has no hostname
    +            // and we want to avoid the reverse lookup, so we return
    +            // the string representation of the address.
    +            if (ia.toString().startsWith("/")) {
    +                hostString = ia.getHostAddress();
    +            } else {
    +                hostString = addr.getHostName();
    +            }
    +        } else {
    +            // According to the Java 6 documentation, if the hostname is
    +            // unresolved, then the string before the colon is the hostname.
    +            String addrString = addr.toString();
    +            hostString = addrString.substring(0, addrString.lastIndexOf(':'));
    +        }
    +
    +        return hostString;
    +    }
    +
         public int size() {
             return serverAddresses.size();
         }
     
    +    // Counts the number of addresses added and removed during
    +    // the last call to next. Used mainly for test purposes.
    +    // See StasticHostProviderTest.
    +    private int nextAdded = 0;
    --- End diff --
    
    instead of exposing all these metrics, which involve making changes to application logic. Why don't we just expose the `serverAddresses` to the test?


---

[GitHub] zookeeper issue #451: ZOOKEEPER-2184: Zookeeper Client should re-resolve hos...

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

    https://github.com/apache/zookeeper/pull/451
  
    @ijuma I feel your pain. :)
    No worries, I'm on it. Doing my best to push committers and others to review changes.
    
    Additionally I'd like to make a small refactoring to the proposed logic before committing, because I'm not entirely convinced about this manual caching/shuffling logic that was implemented originally.
    
    You can see the details in one of my comments above and on the `dev` mailing list.


---

[GitHub] zookeeper issue #451: ZOOKEEPER-2184: Zookeeper Client should re-resolve hos...

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

    https://github.com/apache/zookeeper/pull/451
  
    Looking at the static initialization block in InetAddressCachePolicy more deeply, the default TTL is 30 seconds if there is no SecurityManager installed.
    So caching a positive lookup forever in the Java-level cache is the default only if there is a SecurityManager installed and the TTL is not overridden by "networkaddress.cache.ttl" to a different value.
    Default caching policy for a negative lookup is 0 (never cache).
    Now the only question is whether 30 seconds default caching is ok or too much for ZK.


---

[GitHub] zookeeper pull request #451: ZOOKEEPER-2184: Zookeeper Client should re-reso...

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

    https://github.com/apache/zookeeper/pull/451#discussion_r190846274
  
    --- Diff: src/java/main/org/apache/zookeeper/client/HostProvider.java ---
    @@ -53,7 +54,7 @@
          * @param spinDelay
          *            Milliseconds to wait if all hosts have been tried once.
          */
    -    public InetSocketAddress next(long spinDelay);
    +    public InetSocketAddress next(long spinDelay) throws UnknownHostException;
    --- End diff --
    
    @hanm Answering @fpj comment below:
    
    > > We need a way to break the loop in the case the client closes, though.
    
    > That's actually a good reason for _not_ dealing with the error here. Because the caller - ClientCnxn - is be able to detect client closes, but StatisHostProvider is not.
    
    In a nutshell the problem of unresolvable DNS name must be handled somewhere and here're the considerations:
    1. Client should endlessly try to resolve DNS names instead of giving up at some point. The re-try logic is already implemented in the caller methods properly.
    2. In order to avoid API change we have to replicate the retry logic in the next() method which would add more complexity to the method and more extensive testing
    
    With these considerations I strongly believe that the API change is reasonable.
    That's basically what we discussed in the e-mail thread too.
    
    I'll update the comment as requested. Thanks.


---

[GitHub] zookeeper issue #451: ZOOKEEPER-2184: Zookeeper Client should re-resolve hos...

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

    https://github.com/apache/zookeeper/pull/451
  
    Thanks for picking this up @anmolnar, looking forward to this being fixed. :)


---

[GitHub] zookeeper pull request #451: ZOOKEEPER-2184: Zookeeper Client should re-reso...

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

    https://github.com/apache/zookeeper/pull/451#discussion_r165658743
  
    --- Diff: src/java/main/org/apache/zookeeper/client/StaticHostProvider.java ---
    @@ -25,6 +25,8 @@
     import java.util.Collection;
     import java.util.Collections;
     import java.util.List;
    +import java.lang.reflect.InvocationTargetException;
    --- End diff --
    
    Good catch, thanks.


---

[GitHub] zookeeper pull request #451: ZOOKEEPER-2184: Zookeeper Client should re-reso...

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

    https://github.com/apache/zookeeper/pull/451#discussion_r168564981
  
    --- Diff: src/java/main/org/apache/zookeeper/client/StaticHostProvider.java ---
    @@ -18,6 +18,10 @@
     
     package org.apache.zookeeper.client;
     
    +import org.apache.yetus.audience.InterfaceAudience;
    --- End diff --
    
    nit: move this back


---

[GitHub] zookeeper issue #451: ZOOKEEPER-2184: Zookeeper Client should re-resolve hos...

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

    https://github.com/apache/zookeeper/pull/451
  
    Thanks @anmolnar, @fpj and @hanm for getting this in.


---

[GitHub] zookeeper pull request #451: ZOOKEEPER-2184: Zookeeper Client should re-reso...

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

    https://github.com/apache/zookeeper/pull/451#discussion_r186735649
  
    --- Diff: src/java/main/org/apache/zookeeper/client/HostProvider.java ---
    @@ -53,7 +54,7 @@
          * @param spinDelay
          *            Milliseconds to wait if all hosts have been tried once.
          */
    -    public InetSocketAddress next(long spinDelay);
    +    public InetSocketAddress next(long spinDelay) throws UnknownHostException;
    --- End diff --
    
    1- See my comment below. Now I think the API change is necessary, but I need to double check how it was detected previously.
    2- I started a thread on the mailing list a while ago for exactly the same reason. Let me update it.


---

[GitHub] zookeeper pull request #451: ZOOKEEPER-2184: Zookeeper Client should re-reso...

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

    https://github.com/apache/zookeeper/pull/451#discussion_r190789369
  
    --- Diff: src/java/main/org/apache/zookeeper/client/HostProvider.java ---
    @@ -53,7 +54,7 @@
          * @param spinDelay
          *            Milliseconds to wait if all hosts have been tried once.
          */
    -    public InetSocketAddress next(long spinDelay);
    +    public InetSocketAddress next(long spinDelay) throws UnknownHostException;
    --- End diff --
    
    Some questions here @anmolnar:
    
    >> See my comment below. Now I think the API change is necessary, but I need to double check how it was detected previously.
    
    Which comment? I had a hard time find it, could you please explicitly paste it here?
    
    >> I started a thread on the mailing list a while ago for exactly the same reason. Let me update it.
    
    Similarly I lost that email thread. Could you please describe the gist of the email?
    
    I just wan to make sure we all agree that first an API change here is needed and second, if the API changes we need throw the right type of exception. 
    
    (Also note the current comment of HostProvider interface should be updated if the API changes, because previous semantic of next always assume returns a "valid" address.).


---