New issue
Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.
By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.
Already on GitHub? Sign in to your account
KAFKA-12193: Re-resolve IPs after a client disconnects #9902
Changes from 1 commit
File filter
Filter by extension
Conversations
Jump to
Diff view
Diff view
There are no files selected for viewing
Original file line number | Diff line number | Diff line change |
---|---|---|
|
@@ -43,13 +43,14 @@ final class ClusterConnectionStates { | |
final static double CONNECTION_SETUP_TIMEOUT_JITTER = 0.2; | ||
private final Map<String, NodeConnectionState> nodeState; | ||
private final Logger log; | ||
private final HostResolver hostResolver; | ||
private Set<String> connectingNodes; | ||
private ExponentialBackoff reconnectBackoff; | ||
private ExponentialBackoff connectionSetupTimeout; | ||
|
||
public ClusterConnectionStates(long reconnectBackoffMs, long reconnectBackoffMaxMs, | ||
long connectionSetupTimeoutMs, long connectionSetupTimeoutMaxMs, | ||
LogContext logContext) { | ||
LogContext logContext, HostResolver hostResolver) { | ||
this.log = logContext.logger(ClusterConnectionStates.class); | ||
this.reconnectBackoff = new ExponentialBackoff( | ||
reconnectBackoffMs, | ||
|
@@ -63,6 +64,7 @@ public ClusterConnectionStates(long reconnectBackoffMs, long reconnectBackoffMax | |
CONNECTION_SETUP_TIMEOUT_JITTER); | ||
this.nodeState = new HashMap<>(); | ||
this.connectingNodes = new HashSet<>(); | ||
this.hostResolver = hostResolver; | ||
} | ||
|
||
/** | ||
|
@@ -156,7 +158,8 @@ public void connecting(String id, long now, String host, ClientDnsLookup clientD | |
// Create a new NodeConnectionState if nodeState does not already contain one | ||
// for the specified id or if the hostname associated with the node id changed. | ||
nodeState.put(id, new NodeConnectionState(ConnectionState.CONNECTING, now, | ||
reconnectBackoff.backoff(0), connectionSetupTimeout.backoff(0), host, clientDnsLookup)); | ||
reconnectBackoff.backoff(0), connectionSetupTimeout.backoff(0), host, | ||
clientDnsLookup, hostResolver)); | ||
connectingNodes.add(id); | ||
} | ||
|
||
|
@@ -183,6 +186,10 @@ public void disconnected(String id, long now) { | |
connectingNodes.remove(id); | ||
} else { | ||
resetConnectionSetupTimeout(nodeState); | ||
if (nodeState.state.isConnected()) { | ||
// If a connection had previously been established, re-resolve DNS because the IPs may have changed | ||
nodeState.addresses = Collections.emptyList(); | ||
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. Slightly weird that we're updating a "private" field here. Also the comment is a bit misleading. We're not re-resolving DNS here but instead clearing state so if we reconnect later, the client will be forced to re-resolve then. There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. @mimaison Java allows outer classes to access inner member's private members. This is described in detail at JLS-6.6.1
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. Agree with @mimaison on the comment, good to make it clear by mentioning that the addresses are cleared to re-resolve later when it reconnects. There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. Accessing the field is allowed, but I think @mimaison meant that it makes the code a little confusing and inconsistent. Agree about the comment, I've updated it |
||
} | ||
} | ||
nodeState.state = ConnectionState.DISCONNECTED; | ||
} | ||
|
@@ -470,9 +477,11 @@ private static class NodeConnectionState { | |
private int addressIndex; | ||
private final String host; | ||
private final ClientDnsLookup clientDnsLookup; | ||
private final HostResolver hostResolver; | ||
|
||
private NodeConnectionState(ConnectionState state, long lastConnectAttempt, long reconnectBackoffMs, | ||
long connectionSetupTimeoutMs, String host, ClientDnsLookup clientDnsLookup) { | ||
long connectionSetupTimeoutMs, String host, ClientDnsLookup clientDnsLookup, | ||
HostResolver hostResolver) { | ||
this.state = state; | ||
this.addresses = Collections.emptyList(); | ||
this.addressIndex = -1; | ||
|
@@ -484,6 +493,7 @@ private NodeConnectionState(ConnectionState state, long lastConnectAttempt, long | |
this.throttleUntilTimeMs = 0; | ||
this.host = host; | ||
this.clientDnsLookup = clientDnsLookup; | ||
this.hostResolver = hostResolver; | ||
} | ||
|
||
public String host() { | ||
|
@@ -498,7 +508,7 @@ public String host() { | |
private InetAddress currentAddress() throws UnknownHostException { | ||
if (addresses.isEmpty()) { | ||
// (Re-)initialize list | ||
addresses = ClientUtils.resolve(host, clientDnsLookup); | ||
addresses = ClientUtils.resolve(host, clientDnsLookup, hostResolver); | ||
addressIndex = 0; | ||
} | ||
|
||
|
Original file line number | Diff line number | Diff line change |
---|---|---|
@@ -0,0 +1,29 @@ | ||
/* | ||
* 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. | ||
*/ | ||
|
||
package org.apache.kafka.clients; | ||
|
||
import java.net.InetAddress; | ||
import java.net.UnknownHostException; | ||
|
||
public class DefaultHostResolver implements HostResolver { | ||
|
||
@Override | ||
public InetAddress[] resolve(String host) throws UnknownHostException { | ||
return InetAddress.getAllByName(host); | ||
} | ||
} | ||
bob-barrett marked this conversation as resolved.
Show resolved
Hide resolved
|
Original file line number | Diff line number | Diff line change |
---|---|---|
@@ -0,0 +1,26 @@ | ||
/* | ||
* 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. | ||
*/ | ||
|
||
package org.apache.kafka.clients; | ||
|
||
import java.net.InetAddress; | ||
import java.net.UnknownHostException; | ||
|
||
public interface HostResolver { | ||
|
||
InetAddress[] resolve(String host) throws UnknownHostException; | ||
} | ||
bob-barrett marked this conversation as resolved.
Show resolved
Hide resolved
|
Original file line number | Diff line number | Diff line change |
---|---|---|
|
@@ -130,6 +130,8 @@ private enum State { | |
|
||
private final AtomicReference<State> state; | ||
|
||
private final HostResolver hostResolver; | ||
|
||
public NetworkClient(Selectable selector, | ||
Metadata metadata, | ||
String clientId, | ||
|
@@ -239,24 +241,63 @@ public NetworkClient(Selectable selector, | |
logContext); | ||
} | ||
|
||
private NetworkClient(MetadataUpdater metadataUpdater, | ||
Metadata metadata, | ||
Selectable selector, | ||
String clientId, | ||
int maxInFlightRequestsPerConnection, | ||
long reconnectBackoffMs, | ||
long reconnectBackoffMax, | ||
int socketSendBuffer, | ||
int socketReceiveBuffer, | ||
int defaultRequestTimeoutMs, | ||
long connectionSetupTimeoutMs, | ||
long connectionSetupTimeoutMaxMs, | ||
ClientDnsLookup clientDnsLookup, | ||
Time time, | ||
boolean discoverBrokerVersions, | ||
ApiVersions apiVersions, | ||
Sensor throttleTimeSensor, | ||
LogContext logContext) { | ||
public NetworkClient(MetadataUpdater metadataUpdater, | ||
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. Do we really need this constructor? As far as I can tell, it's only called by the other 3 above. These could directly call the real one below instead of going through this new one. WDYT? There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. Good call, fixed. These are also weirdly inconsistent ( |
||
Metadata metadata, | ||
Selectable selector, | ||
String clientId, | ||
int maxInFlightRequestsPerConnection, | ||
long reconnectBackoffMs, | ||
long reconnectBackoffMax, | ||
int socketSendBuffer, | ||
int socketReceiveBuffer, | ||
int defaultRequestTimeoutMs, | ||
long connectionSetupTimeoutMs, | ||
long connectionSetupTimeoutMaxMs, | ||
ClientDnsLookup clientDnsLookup, | ||
Time time, | ||
boolean discoverBrokerVersions, | ||
ApiVersions apiVersions, | ||
Sensor throttleTimeSensor, | ||
LogContext logContext) { | ||
this(metadataUpdater, | ||
metadata, | ||
selector, | ||
clientId, maxInFlightRequestsPerConnection, | ||
reconnectBackoffMs, | ||
reconnectBackoffMax, | ||
socketSendBuffer, | ||
socketReceiveBuffer, | ||
defaultRequestTimeoutMs, | ||
connectionSetupTimeoutMs, | ||
connectionSetupTimeoutMaxMs, | ||
clientDnsLookup, | ||
time, | ||
discoverBrokerVersions, | ||
apiVersions, | ||
throttleTimeSensor, | ||
logContext, | ||
new DefaultHostResolver()); | ||
} | ||
|
||
public NetworkClient(MetadataUpdater metadataUpdater, | ||
Metadata metadata, | ||
Selectable selector, | ||
String clientId, | ||
int maxInFlightRequestsPerConnection, | ||
long reconnectBackoffMs, | ||
long reconnectBackoffMax, | ||
int socketSendBuffer, | ||
int socketReceiveBuffer, | ||
int defaultRequestTimeoutMs, | ||
long connectionSetupTimeoutMs, | ||
long connectionSetupTimeoutMaxMs, | ||
ClientDnsLookup clientDnsLookup, | ||
Time time, | ||
boolean discoverBrokerVersions, | ||
ApiVersions apiVersions, | ||
Sensor throttleTimeSensor, | ||
LogContext logContext, | ||
HostResolver hostResolver) { | ||
/* It would be better if we could pass `DefaultMetadataUpdater` from the public constructor, but it's not | ||
* possible because `DefaultMetadataUpdater` is an inner class and it can only be instantiated after the | ||
* super constructor is invoked. | ||
|
@@ -273,7 +314,7 @@ private NetworkClient(MetadataUpdater metadataUpdater, | |
this.inFlightRequests = new InFlightRequests(maxInFlightRequestsPerConnection); | ||
this.connectionStates = new ClusterConnectionStates( | ||
reconnectBackoffMs, reconnectBackoffMax, | ||
connectionSetupTimeoutMs, connectionSetupTimeoutMaxMs, logContext); | ||
connectionSetupTimeoutMs, connectionSetupTimeoutMaxMs, logContext, hostResolver); | ||
this.socketSendBuffer = socketSendBuffer; | ||
this.socketReceiveBuffer = socketReceiveBuffer; | ||
this.correlation = 0; | ||
|
@@ -287,6 +328,7 @@ private NetworkClient(MetadataUpdater metadataUpdater, | |
this.log = logContext.logger(NetworkClient.class); | ||
this.clientDnsLookup = clientDnsLookup; | ||
this.state = new AtomicReference<>(State.ACTIVE); | ||
this.hostResolver = hostResolver; | ||
} | ||
|
||
/** | ||
|
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
This method and its usages look a bit weird to me now. It resolves an
host
based onclientDnsLookup
and ahostResolver
which resolves anhost
at its turn. We also have to pass theHostResolver
wherever it is called now. I wonder if we should avoid this level of indirection here and directly encapsulate the entire logic in theHostResolver
.HostResolver
would look like this:In
NodeConnectionState#currentAddress
, we could replaceaddresses = ClientUtils.resolve(host, clientDnsLookup, hostResolver)
byaddresses = hostResolver.resolve(host, clientDnsLookup)
.The
DefaultHostResolver
would do whatClientUtils.resolve
does today and we can mock it in tests like you did.Is this something that you have considered?
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
@dajac I started making this change, but while working on it, I realized that I don't think I like requiring the
HostResolver
to understand theclient.dns.lookup
config. In the same way thatMockTime
is only used to mock Java built-in methods, I think theHostResolver
should only be responsible for the DNS resolution (either real or mocked), and thatClientUtils
should be responsible for understanding the Kafka application behavior. That way, tests that mock DNS still test the actualClientDnsLookup
behavior, rather than relying on both the default resolver and any mocked resolvers handle it correctly. It also means that if we ever add an option forClientDnsLookup
or change its behavior, we don't need to remember to update allHostResolver
implementations.There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Thanks @bob-barrett. I do agree with your point. Let's keep it as you suggested.