You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@hc.apache.org by ol...@apache.org on 2021/09/12 16:24:56 UTC

[httpcomponents-client] 01/02: Extracted multi-home socket connect logic into a separate utility class; made more consistent with async multi-home connect code

This is an automated email from the ASF dual-hosted git repository.

olegk pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/httpcomponents-client.git

commit ce67fa635a6e5b40550ded122cffe6f6e74d6b60
Author: Oleg Kalnichevski <ol...@apache.org>
AuthorDate: Sun Sep 12 13:56:59 2021 +0200

    Extracted multi-home socket connect logic into a separate utility class; made more consistent with async multi-home connect code
---
 .../impl/io/BasicHttpClientConnectionManager.java  |   6 ++
 .../io/DefaultHttpClientConnectionOperator.java    |  76 ++++----------
 .../http/impl/io/MultihomeSocketConnector.java     | 111 +++++++++++++++++++++
 .../http/impl/nio/MultihomeIOSessionRequester.java |   4 +-
 .../impl/io/TestHttpClientConnectionOperator.java  |   2 +-
 5 files changed, 141 insertions(+), 58 deletions(-)

diff --git a/httpclient5/src/main/java/org/apache/hc/client5/http/impl/io/BasicHttpClientConnectionManager.java b/httpclient5/src/main/java/org/apache/hc/client5/http/impl/io/BasicHttpClientConnectionManager.java
index a529dfb..b9866c1 100644
--- a/httpclient5/src/main/java/org/apache/hc/client5/http/impl/io/BasicHttpClientConnectionManager.java
+++ b/httpclient5/src/main/java/org/apache/hc/client5/http/impl/io/BasicHttpClientConnectionManager.java
@@ -361,6 +361,9 @@ public class BasicHttpClientConnectionManager implements HttpClientConnectionMan
         final ConnectionConfig config = connectionConfig != null ? connectionConfig : ConnectionConfig.DEFAULT;
         final TimeValue connectTimeout = timeout != null ? timeout : config.getConnectTimeout();
         final ManagedHttpClientConnection connection = internalEndpoint.getConnection();
+        if (LOG.isDebugEnabled()) {
+            LOG.debug("{} connecting endpoint to {} ({})", ConnPoolSupport.getId(endpoint), host, connectTimeout);
+        }
         this.connectionOperator.connect(
                 connection,
                 host,
@@ -368,6 +371,9 @@ public class BasicHttpClientConnectionManager implements HttpClientConnectionMan
                 connectTimeout,
                 this.socketConfig,
                 context);
+        if (LOG.isDebugEnabled()) {
+            LOG.debug("{} connected {}", ConnPoolSupport.getId(endpoint), ConnPoolSupport.getId(conn));
+        }
         final Timeout socketTimeout = config.getSocketTimeout();
         if (socketTimeout != null) {
             connection.setSocketTimeout(socketTimeout);
diff --git a/httpclient5/src/main/java/org/apache/hc/client5/http/impl/io/DefaultHttpClientConnectionOperator.java b/httpclient5/src/main/java/org/apache/hc/client5/http/impl/io/DefaultHttpClientConnectionOperator.java
index 14a45f0..27b156e 100644
--- a/httpclient5/src/main/java/org/apache/hc/client5/http/impl/io/DefaultHttpClientConnectionOperator.java
+++ b/httpclient5/src/main/java/org/apache/hc/client5/http/impl/io/DefaultHttpClientConnectionOperator.java
@@ -27,16 +27,13 @@
 package org.apache.hc.client5.http.impl.io;
 
 import java.io.IOException;
-import java.net.InetAddress;
 import java.net.InetSocketAddress;
 import java.net.Socket;
 
-import org.apache.hc.client5.http.ConnectExceptionSupport;
 import org.apache.hc.client5.http.DnsResolver;
 import org.apache.hc.client5.http.SchemePortResolver;
 import org.apache.hc.client5.http.SystemDefaultDnsResolver;
 import org.apache.hc.client5.http.UnsupportedSchemeException;
-import org.apache.hc.client5.http.impl.ConnPoolSupport;
 import org.apache.hc.client5.http.impl.DefaultSchemePortResolver;
 import org.apache.hc.client5.http.io.HttpClientConnectionOperator;
 import org.apache.hc.client5.http.io.ManagedHttpClientConnection;
@@ -53,8 +50,6 @@ import org.apache.hc.core5.http.io.SocketConfig;
 import org.apache.hc.core5.http.protocol.HttpContext;
 import org.apache.hc.core5.util.Args;
 import org.apache.hc.core5.util.TimeValue;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
 
 /**
  * Default implementation of {@link HttpClientConnectionOperator} used as default in Http client,
@@ -69,11 +64,9 @@ public class DefaultHttpClientConnectionOperator implements HttpClientConnection
 
     static final String SOCKET_FACTORY_REGISTRY = "http.socket-factory-registry";
 
-    private static final Logger LOG = LoggerFactory.getLogger(DefaultHttpClientConnectionOperator.class);
-
     private final Lookup<ConnectionSocketFactory> socketFactoryRegistry;
     private final SchemePortResolver schemePortResolver;
-    private final DnsResolver dnsResolver;
+    private final MultihomeSocketConnector multihomeSocketConnector;
 
     public DefaultHttpClientConnectionOperator(
             final Lookup<ConnectionSocketFactory> socketFactoryRegistry,
@@ -82,10 +75,10 @@ public class DefaultHttpClientConnectionOperator implements HttpClientConnection
         super();
         Args.notNull(socketFactoryRegistry, "Socket factory registry");
         this.socketFactoryRegistry = socketFactoryRegistry;
-        this.schemePortResolver = schemePortResolver != null ? schemePortResolver :
-            DefaultSchemePortResolver.INSTANCE;
-        this.dnsResolver = dnsResolver != null ? dnsResolver :
-            SystemDefaultDnsResolver.INSTANCE;
+        this.schemePortResolver = schemePortResolver != null ? schemePortResolver : DefaultSchemePortResolver.INSTANCE;
+        this.multihomeSocketConnector = new MultihomeSocketConnector(
+                schemePortResolver != null ? schemePortResolver : DefaultSchemePortResolver.INSTANCE,
+                dnsResolver != null ? dnsResolver : SystemDefaultDnsResolver.INSTANCE);
     }
 
     @SuppressWarnings("unchecked")
@@ -115,51 +108,26 @@ public class DefaultHttpClientConnectionOperator implements HttpClientConnection
         if (sf == null) {
             throw new UnsupportedSchemeException(host.getSchemeName() + " protocol is not supported");
         }
-        final InetAddress[] addresses = host.getAddress() != null ?
-                new InetAddress[] { host.getAddress() } : this.dnsResolver.resolve(host.getHostName());
-        final int port = this.schemePortResolver.resolve(host);
-        for (int i = 0; i < addresses.length; i++) {
-            final InetAddress address = addresses[i];
-            final boolean last = i == addresses.length - 1;
-
-            Socket sock = sf.createSocket(context);
-            sock.setSoTimeout(socketConfig.getSoTimeout().toMillisecondsIntBound());
-            sock.setReuseAddress(socketConfig.isSoReuseAddress());
-            sock.setTcpNoDelay(socketConfig.isTcpNoDelay());
-            sock.setKeepAlive(socketConfig.isSoKeepAlive());
-            if (socketConfig.getRcvBufSize() > 0) {
-                sock.setReceiveBufferSize(socketConfig.getRcvBufSize());
-            }
-            if (socketConfig.getSndBufSize() > 0) {
-                sock.setSendBufferSize(socketConfig.getSndBufSize());
-            }
+        final Socket sock = sf.createSocket(context);
+        conn.bind(sock);
 
-            final int linger = socketConfig.getSoLinger().toMillisecondsIntBound();
-            if (linger >= 0) {
-                sock.setSoLinger(true, linger);
-            }
-            conn.bind(sock);
+        sock.setSoTimeout(socketConfig.getSoTimeout().toMillisecondsIntBound());
+        sock.setReuseAddress(socketConfig.isSoReuseAddress());
+        sock.setTcpNoDelay(socketConfig.isTcpNoDelay());
+        sock.setKeepAlive(socketConfig.isSoKeepAlive());
+        if (socketConfig.getRcvBufSize() > 0) {
+            sock.setReceiveBufferSize(socketConfig.getRcvBufSize());
+        }
+        if (socketConfig.getSndBufSize() > 0) {
+            sock.setSendBufferSize(socketConfig.getSndBufSize());
+        }
 
-            final InetSocketAddress remoteAddress = new InetSocketAddress(address, port);
-            if (LOG.isDebugEnabled()) {
-                LOG.debug("{} connecting to {}", ConnPoolSupport.getId(conn), remoteAddress);
-            }
-            try {
-                sock = sf.connectSocket(connectTimeout, sock, host, remoteAddress, localAddress, context);
-                conn.bind(sock);
-                if (LOG.isDebugEnabled()) {
-                    LOG.debug("{} connection established {}", ConnPoolSupport.getId(conn), conn);
-                }
-                return;
-            } catch (final IOException ex) {
-                if (last) {
-                    throw ConnectExceptionSupport.enhance(ex, host, addresses);
-                }
-            }
-            if (LOG.isDebugEnabled()) {
-                LOG.debug("{} connect to {} timed out. Connection will be retried using another IP address", ConnPoolSupport.getId(conn), remoteAddress);
-            }
+        final int linger = socketConfig.getSoLinger().toMillisecondsIntBound();
+        if (linger >= 0) {
+            sock.setSoLinger(true, linger);
         }
+        final Socket connectedSocket = multihomeSocketConnector.connect(sock, host, localAddress, connectTimeout, sf, context);
+        conn.bind(connectedSocket);
     }
 
     @Override
diff --git a/httpclient5/src/main/java/org/apache/hc/client5/http/impl/io/MultihomeSocketConnector.java b/httpclient5/src/main/java/org/apache/hc/client5/http/impl/io/MultihomeSocketConnector.java
new file mode 100644
index 0000000..a80d22e
--- /dev/null
+++ b/httpclient5/src/main/java/org/apache/hc/client5/http/impl/io/MultihomeSocketConnector.java
@@ -0,0 +1,111 @@
+/*
+ * ====================================================================
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * 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
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ * ====================================================================
+ *
+ * This software consists of voluntary contributions made by many
+ * individuals on behalf of the Apache Software Foundation.  For more
+ * information on the Apache Software Foundation, please see
+ * <http://www.apache.org/>.
+ *
+ */
+package org.apache.hc.client5.http.impl.io;
+
+import java.io.IOException;
+import java.net.ConnectException;
+import java.net.InetAddress;
+import java.net.InetSocketAddress;
+import java.net.Socket;
+import java.util.Arrays;
+
+import org.apache.hc.client5.http.ConnectExceptionSupport;
+import org.apache.hc.client5.http.DnsResolver;
+import org.apache.hc.client5.http.SchemePortResolver;
+import org.apache.hc.client5.http.socket.ConnectionSocketFactory;
+import org.apache.hc.core5.http.HttpHost;
+import org.apache.hc.core5.http.protocol.HttpContext;
+import org.apache.hc.core5.util.TimeValue;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+final class MultihomeSocketConnector {
+
+    private static final Logger LOG = LoggerFactory.getLogger(MultihomeSocketConnector.class);
+
+    private final SchemePortResolver schemePortResolver;
+    private final DnsResolver dnsResolver;
+
+    MultihomeSocketConnector(final SchemePortResolver schemePortResolver, final DnsResolver dnsResolver) {
+        this.schemePortResolver = schemePortResolver;
+        this.dnsResolver = dnsResolver;
+    }
+
+    public Socket connect(final Socket socket,
+                          final HttpHost host,
+                          final InetSocketAddress localAddress,
+                          final TimeValue connectTimeout,
+                          final ConnectionSocketFactory connectionSocketFactory,
+                          final HttpContext context) throws IOException {
+        final int port = this.schemePortResolver.resolve(host);
+        if (host.getAddress() != null) {
+            final InetSocketAddress remoteAddress = new InetSocketAddress(host.getAddress(), port);
+            if (LOG.isDebugEnabled()) {
+                LOG.debug("{} connecting {} to {} ({})", host, localAddress, remoteAddress, connectTimeout);
+            }
+            final Socket connectedSocket = connectionSocketFactory.connectSocket(connectTimeout, socket, host, remoteAddress, localAddress, context);
+            if (LOG.isDebugEnabled()) {
+                LOG.debug("{} connected {}->{}", host, connectedSocket.getLocalSocketAddress(), connectedSocket.getRemoteSocketAddress());
+            }
+            return connectedSocket;
+        }
+
+        if (LOG.isDebugEnabled()) {
+            LOG.debug("{} resolving remote address", host);
+        }
+
+        final InetAddress[] remoteAddresses = this.dnsResolver.resolve(host.getHostName());
+
+        if (LOG.isDebugEnabled()) {
+            LOG.debug("{} resolved to {}", host, Arrays.asList(remoteAddresses));
+        }
+
+        for (int i = 0; i < remoteAddresses.length; i++) {
+            final InetSocketAddress remoteAddress = new InetSocketAddress(remoteAddresses[i], port);
+            final boolean last = i == remoteAddresses.length - 1;
+            if (LOG.isDebugEnabled()) {
+                LOG.debug("{} connecting to {}", host, remoteAddress);
+            }
+            try {
+                final Socket connectedSocket = connectionSocketFactory.connectSocket(connectTimeout, socket, host, remoteAddress, localAddress, context);
+                if (LOG.isDebugEnabled()) {
+                    LOG.debug("{} connected {}->{}", host, connectedSocket.getLocalSocketAddress(), connectedSocket.getRemoteSocketAddress());
+                }
+                return connectedSocket;
+            } catch (final IOException ex) {
+                if (last) {
+                    throw ConnectExceptionSupport.enhance(ex, host, remoteAddresses);
+                }
+                if (LOG.isDebugEnabled()) {
+                    LOG.debug("{} connection to {} failed ({}); retrying connection to the next address", host, remoteAddress, ex.getClass());
+                }
+            }
+        }
+        throw ConnectExceptionSupport.enhance(new ConnectException(), host, remoteAddresses);
+    }
+
+}
diff --git a/httpclient5/src/main/java/org/apache/hc/client5/http/impl/nio/MultihomeIOSessionRequester.java b/httpclient5/src/main/java/org/apache/hc/client5/http/impl/nio/MultihomeIOSessionRequester.java
index 4d90b85..e5b70e9 100644
--- a/httpclient5/src/main/java/org/apache/hc/client5/http/impl/nio/MultihomeIOSessionRequester.java
+++ b/httpclient5/src/main/java/org/apache/hc/client5/http/impl/nio/MultihomeIOSessionRequester.java
@@ -113,9 +113,7 @@ final class MultihomeIOSessionRequester {
                             @Override
                             public void completed(final IOSession session) {
                                 if (LOG.isDebugEnabled()) {
-                                    if (LOG.isDebugEnabled()) {
-                                        LOG.debug("{} connected {} {}->{}", remoteEndpoint, session.getId(), session.getLocalAddress(), session.getRemoteAddress());
-                                    }
+                                    LOG.debug("{} connected {} {}->{}", remoteEndpoint, session.getId(), session.getLocalAddress(), session.getRemoteAddress());
                                 }
                                 future.completed(session);
                             }
diff --git a/httpclient5/src/test/java/org/apache/hc/client5/http/impl/io/TestHttpClientConnectionOperator.java b/httpclient5/src/test/java/org/apache/hc/client5/http/impl/io/TestHttpClientConnectionOperator.java
index 34bb77a..6becde8 100644
--- a/httpclient5/src/test/java/org/apache/hc/client5/http/impl/io/TestHttpClientConnectionOperator.java
+++ b/httpclient5/src/test/java/org/apache/hc/client5/http/impl/io/TestHttpClientConnectionOperator.java
@@ -209,7 +209,7 @@ public class TestHttpClientConnectionOperator {
                 new InetSocketAddress(ip2, 80),
                 localAddress,
                 context);
-        Mockito.verify(conn, Mockito.times(3)).bind(socket);
+        Mockito.verify(conn, Mockito.times(2)).bind(socket);
     }
 
     @Test