|
|
@ -97,6 +97,7 @@ import java.util.Map; |
|
|
|
import java.util.TreeMap; |
|
|
|
import java.util.TreeMap; |
|
|
|
import java.util.concurrent.TimeUnit; |
|
|
|
import java.util.concurrent.TimeUnit; |
|
|
|
import java.util.concurrent.atomic.AtomicInteger; |
|
|
|
import java.util.concurrent.atomic.AtomicInteger; |
|
|
|
|
|
|
|
import java.util.concurrent.atomic.AtomicLong; |
|
|
|
|
|
|
|
|
|
|
|
import static org.apache.kafka.common.utils.Utils.closeQuietly; |
|
|
|
import static org.apache.kafka.common.utils.Utils.closeQuietly; |
|
|
|
|
|
|
|
|
|
|
@ -123,6 +124,11 @@ public class KafkaAdminClient extends AdminClient { |
|
|
|
*/ |
|
|
|
*/ |
|
|
|
private static final String JMX_PREFIX = "kafka.admin.client"; |
|
|
|
private static final String JMX_PREFIX = "kafka.admin.client"; |
|
|
|
|
|
|
|
|
|
|
|
|
|
|
|
/** |
|
|
|
|
|
|
|
* An invalid shutdown time which indicates that a shutdown has not yet been performed. |
|
|
|
|
|
|
|
*/ |
|
|
|
|
|
|
|
private static final long INVALID_SHUTDOWN_TIME = -1; |
|
|
|
|
|
|
|
|
|
|
|
/** |
|
|
|
/** |
|
|
|
* The default timeout to use for an operation. |
|
|
|
* The default timeout to use for an operation. |
|
|
|
*/ |
|
|
|
*/ |
|
|
@ -164,9 +170,10 @@ public class KafkaAdminClient extends AdminClient { |
|
|
|
private final Thread thread; |
|
|
|
private final Thread thread; |
|
|
|
|
|
|
|
|
|
|
|
/** |
|
|
|
/** |
|
|
|
* True if this client is closed. |
|
|
|
* During a close operation, this is the time at which we will time out all pending operations |
|
|
|
|
|
|
|
* and force the RPC thread to exit. If the admin client is not closing, this will be 0. |
|
|
|
*/ |
|
|
|
*/ |
|
|
|
private volatile boolean closed = false; |
|
|
|
private final AtomicLong hardShutdownTimeMs = new AtomicLong(INVALID_SHUTDOWN_TIME); |
|
|
|
|
|
|
|
|
|
|
|
/** |
|
|
|
/** |
|
|
|
* Get or create a list value from a map. |
|
|
|
* Get or create a list value from a map. |
|
|
@ -289,12 +296,12 @@ public class KafkaAdminClient extends AdminClient { |
|
|
|
selector, |
|
|
|
selector, |
|
|
|
metadata, |
|
|
|
metadata, |
|
|
|
clientId, |
|
|
|
clientId, |
|
|
|
100, |
|
|
|
1, |
|
|
|
config.getLong(AdminClientConfig.RECONNECT_BACKOFF_MS_CONFIG), |
|
|
|
config.getLong(AdminClientConfig.RECONNECT_BACKOFF_MS_CONFIG), |
|
|
|
config.getLong(AdminClientConfig.RECONNECT_BACKOFF_MAX_MS_CONFIG), |
|
|
|
config.getLong(AdminClientConfig.RECONNECT_BACKOFF_MAX_MS_CONFIG), |
|
|
|
config.getInt(AdminClientConfig.SEND_BUFFER_CONFIG), |
|
|
|
config.getInt(AdminClientConfig.SEND_BUFFER_CONFIG), |
|
|
|
config.getInt(AdminClientConfig.RECEIVE_BUFFER_CONFIG), |
|
|
|
config.getInt(AdminClientConfig.RECEIVE_BUFFER_CONFIG), |
|
|
|
config.getInt(AdminClientConfig.REQUEST_TIMEOUT_MS_CONFIG), |
|
|
|
(int) TimeUnit.HOURS.toMillis(1), |
|
|
|
time, |
|
|
|
time, |
|
|
|
true, |
|
|
|
true, |
|
|
|
apiVersions); |
|
|
|
apiVersions); |
|
|
@ -309,7 +316,7 @@ public class KafkaAdminClient extends AdminClient { |
|
|
|
} |
|
|
|
} |
|
|
|
} |
|
|
|
} |
|
|
|
|
|
|
|
|
|
|
|
static KafkaAdminClient create(AdminClientConfig config, KafkaClient client, Metadata metadata) { |
|
|
|
static KafkaAdminClient createInternal(AdminClientConfig config, KafkaClient client, Metadata metadata) { |
|
|
|
Metrics metrics = null; |
|
|
|
Metrics metrics = null; |
|
|
|
Time time = Time.SYSTEM; |
|
|
|
Time time = Time.SYSTEM; |
|
|
|
String clientId = generateClientId(config); |
|
|
|
String clientId = generateClientId(config); |
|
|
@ -343,9 +350,33 @@ public class KafkaAdminClient extends AdminClient { |
|
|
|
} |
|
|
|
} |
|
|
|
|
|
|
|
|
|
|
|
@Override |
|
|
|
@Override |
|
|
|
public void close() { |
|
|
|
public void close(long duration, TimeUnit unit) { |
|
|
|
closed = true; |
|
|
|
long waitTimeMs = unit.toMillis(duration); |
|
|
|
client.wakeup(); // Wake the thread, if it is blocked inside poll().
|
|
|
|
waitTimeMs = Math.min(TimeUnit.DAYS.toMillis(365), waitTimeMs); // Limit the timeout to a year.
|
|
|
|
|
|
|
|
long now = time.milliseconds(); |
|
|
|
|
|
|
|
long newHardShutdownTimeMs = now + waitTimeMs; |
|
|
|
|
|
|
|
long prev = INVALID_SHUTDOWN_TIME; |
|
|
|
|
|
|
|
while (true) { |
|
|
|
|
|
|
|
if (hardShutdownTimeMs.compareAndSet(prev, newHardShutdownTimeMs)) { |
|
|
|
|
|
|
|
if (prev == INVALID_SHUTDOWN_TIME) { |
|
|
|
|
|
|
|
log.debug("{}: initiating close operation.", clientId); |
|
|
|
|
|
|
|
} else { |
|
|
|
|
|
|
|
log.debug("{}: moving hard shutdown time forward.", clientId); |
|
|
|
|
|
|
|
} |
|
|
|
|
|
|
|
client.wakeup(); // Wake the thread, if it is blocked inside poll().
|
|
|
|
|
|
|
|
break; |
|
|
|
|
|
|
|
} |
|
|
|
|
|
|
|
prev = hardShutdownTimeMs.get(); |
|
|
|
|
|
|
|
if (prev < newHardShutdownTimeMs) { |
|
|
|
|
|
|
|
log.debug("{}: hard shutdown time is already earlier than requested.", clientId); |
|
|
|
|
|
|
|
newHardShutdownTimeMs = prev; |
|
|
|
|
|
|
|
break; |
|
|
|
|
|
|
|
} |
|
|
|
|
|
|
|
} |
|
|
|
|
|
|
|
if (log.isDebugEnabled()) { |
|
|
|
|
|
|
|
long deltaMs = Math.max(0, newHardShutdownTimeMs - time.milliseconds()); |
|
|
|
|
|
|
|
log.debug("{}: waiting for the I/O thread to exit. Hard shutdown in {} ms.", clientId, deltaMs); |
|
|
|
|
|
|
|
} |
|
|
|
try { |
|
|
|
try { |
|
|
|
// Wait for the thread to be joined.
|
|
|
|
// Wait for the thread to be joined.
|
|
|
|
thread.join(); |
|
|
|
thread.join(); |
|
|
@ -439,7 +470,7 @@ public class KafkaAdminClient extends AdminClient { |
|
|
|
if ((throwable instanceof UnsupportedVersionException) && |
|
|
|
if ((throwable instanceof UnsupportedVersionException) && |
|
|
|
handleUnsupportedVersionException((UnsupportedVersionException) throwable)) { |
|
|
|
handleUnsupportedVersionException((UnsupportedVersionException) throwable)) { |
|
|
|
log.trace("{} attempting protocol downgrade.", this); |
|
|
|
log.trace("{} attempting protocol downgrade.", this); |
|
|
|
runnable.call(this, now); |
|
|
|
runnable.enqueue(this, now); |
|
|
|
return; |
|
|
|
return; |
|
|
|
} |
|
|
|
} |
|
|
|
tries++; |
|
|
|
tries++; |
|
|
@ -474,7 +505,7 @@ public class KafkaAdminClient extends AdminClient { |
|
|
|
log.debug("{} failed: {}. Beginning retry #{}", |
|
|
|
log.debug("{} failed: {}. Beginning retry #{}", |
|
|
|
this, prettyPrintException(throwable), tries); |
|
|
|
this, prettyPrintException(throwable), tries); |
|
|
|
} |
|
|
|
} |
|
|
|
runnable.call(this, now); |
|
|
|
runnable.enqueue(this, now); |
|
|
|
} |
|
|
|
} |
|
|
|
|
|
|
|
|
|
|
|
/** |
|
|
|
/** |
|
|
@ -523,6 +554,7 @@ public class KafkaAdminClient extends AdminClient { |
|
|
|
private final class AdminClientRunnable implements Runnable { |
|
|
|
private final class AdminClientRunnable implements Runnable { |
|
|
|
/** |
|
|
|
/** |
|
|
|
* Pending calls. Protected by the object monitor. |
|
|
|
* Pending calls. Protected by the object monitor. |
|
|
|
|
|
|
|
* This will be null only if the thread has shut down. |
|
|
|
*/ |
|
|
|
*/ |
|
|
|
private List<Call> newCalls = new LinkedList<>(); |
|
|
|
private List<Call> newCalls = new LinkedList<>(); |
|
|
|
|
|
|
|
|
|
|
@ -554,47 +586,96 @@ public class KafkaAdminClient extends AdminClient { |
|
|
|
return null; |
|
|
|
return null; |
|
|
|
} |
|
|
|
} |
|
|
|
|
|
|
|
|
|
|
|
/** |
|
|
|
private class TimeoutProcessor { |
|
|
|
* Time out a list of calls. |
|
|
|
/** |
|
|
|
* |
|
|
|
* The current time in milliseconds. |
|
|
|
* @param now The current time in milliseconds. |
|
|
|
*/ |
|
|
|
* @param calls The collection of calls. Must be sorted from oldest to newest. |
|
|
|
private final long now; |
|
|
|
*/ |
|
|
|
|
|
|
|
private int timeoutCalls(long now, Collection<Call> calls) { |
|
|
|
/** |
|
|
|
int numTimedOut = 0; |
|
|
|
* The number of milliseconds until the next timeout. |
|
|
|
for (Iterator<Call> iter = calls.iterator(); iter.hasNext(); ) { |
|
|
|
*/ |
|
|
|
Call call = iter.next(); |
|
|
|
private int nextTimeoutMs; |
|
|
|
if (calcTimeoutMsRemainingAsInt(now, call.deadlineMs) < 0) { |
|
|
|
|
|
|
|
call.fail(now, new TimeoutException()); |
|
|
|
/** |
|
|
|
iter.remove(); |
|
|
|
* Create a new timeout processor. |
|
|
|
numTimedOut++; |
|
|
|
* |
|
|
|
|
|
|
|
* @param now The current time in milliseconds since the epoch. |
|
|
|
|
|
|
|
*/ |
|
|
|
|
|
|
|
TimeoutProcessor(long now) { |
|
|
|
|
|
|
|
this.now = now; |
|
|
|
|
|
|
|
this.nextTimeoutMs = Integer.MAX_VALUE; |
|
|
|
|
|
|
|
} |
|
|
|
|
|
|
|
|
|
|
|
|
|
|
|
/** |
|
|
|
|
|
|
|
* Check for calls which have timed out. |
|
|
|
|
|
|
|
* Timed out calls will be removed and failed. |
|
|
|
|
|
|
|
* The remaining milliseconds until the next timeout will be updated. |
|
|
|
|
|
|
|
* |
|
|
|
|
|
|
|
* @param calls The collection of calls. |
|
|
|
|
|
|
|
* |
|
|
|
|
|
|
|
* @return The number of calls which were timed out. |
|
|
|
|
|
|
|
*/ |
|
|
|
|
|
|
|
int handleTimeouts(Collection<Call> calls, String msg) { |
|
|
|
|
|
|
|
int numTimedOut = 0; |
|
|
|
|
|
|
|
for (Iterator<Call> iter = calls.iterator(); iter.hasNext(); ) { |
|
|
|
|
|
|
|
Call call = iter.next(); |
|
|
|
|
|
|
|
int remainingMs = calcTimeoutMsRemainingAsInt(now, call.deadlineMs); |
|
|
|
|
|
|
|
if (remainingMs < 0) { |
|
|
|
|
|
|
|
call.fail(now, new TimeoutException(msg)); |
|
|
|
|
|
|
|
iter.remove(); |
|
|
|
|
|
|
|
numTimedOut++; |
|
|
|
|
|
|
|
} else { |
|
|
|
|
|
|
|
nextTimeoutMs = Math.min(nextTimeoutMs, remainingMs); |
|
|
|
|
|
|
|
} |
|
|
|
} |
|
|
|
} |
|
|
|
|
|
|
|
return numTimedOut; |
|
|
|
|
|
|
|
} |
|
|
|
|
|
|
|
|
|
|
|
|
|
|
|
/** |
|
|
|
|
|
|
|
* Check whether a call should be timed out. |
|
|
|
|
|
|
|
* The remaining milliseconds until the next timeout will be updated. |
|
|
|
|
|
|
|
* |
|
|
|
|
|
|
|
* @param call The call. |
|
|
|
|
|
|
|
* |
|
|
|
|
|
|
|
* @return True if the call should be timed out. |
|
|
|
|
|
|
|
*/ |
|
|
|
|
|
|
|
boolean callHasExpired(Call call) { |
|
|
|
|
|
|
|
int remainingMs = calcTimeoutMsRemainingAsInt(now, call.deadlineMs); |
|
|
|
|
|
|
|
if (remainingMs < 0) |
|
|
|
|
|
|
|
return true; |
|
|
|
|
|
|
|
nextTimeoutMs = Math.min(nextTimeoutMs, remainingMs); |
|
|
|
|
|
|
|
return false; |
|
|
|
|
|
|
|
} |
|
|
|
|
|
|
|
|
|
|
|
|
|
|
|
int nextTimeoutMs() { |
|
|
|
|
|
|
|
return nextTimeoutMs; |
|
|
|
} |
|
|
|
} |
|
|
|
return numTimedOut; |
|
|
|
|
|
|
|
} |
|
|
|
} |
|
|
|
|
|
|
|
|
|
|
|
/** |
|
|
|
/** |
|
|
|
* Time out the elements in the newCalls list which are expired. |
|
|
|
* Time out the elements in the newCalls list which are expired. |
|
|
|
* |
|
|
|
* |
|
|
|
* @param now The current time in milliseconds. |
|
|
|
* @param processor The timeout processor. |
|
|
|
*/ |
|
|
|
*/ |
|
|
|
private synchronized void timeoutNewCalls(long now) { |
|
|
|
private synchronized void timeoutNewCalls(TimeoutProcessor processor) { |
|
|
|
int numTimedOut = timeoutCalls(now, newCalls); |
|
|
|
int numTimedOut = processor.handleTimeouts(newCalls, |
|
|
|
if (numTimedOut > 0) { |
|
|
|
"Timed out waiting for a node assignment."); |
|
|
|
|
|
|
|
if (numTimedOut > 0) |
|
|
|
log.debug("{}: timed out {} new calls.", clientId, numTimedOut); |
|
|
|
log.debug("{}: timed out {} new calls.", clientId, numTimedOut); |
|
|
|
} |
|
|
|
|
|
|
|
} |
|
|
|
} |
|
|
|
|
|
|
|
|
|
|
|
/** |
|
|
|
/** |
|
|
|
* Time out calls which have been assigned to nodes. |
|
|
|
* Time out calls which have been assigned to nodes. |
|
|
|
* |
|
|
|
* |
|
|
|
* @param now The current time in milliseconds. |
|
|
|
* @param processor The timeout processor. |
|
|
|
* @param callsToSend A map of nodes to the calls they need to handle. |
|
|
|
* @param callsToSend A map of nodes to the calls they need to handle. |
|
|
|
*/ |
|
|
|
*/ |
|
|
|
private void timeoutCallsToSend(long now, Map<Node, List<Call>> callsToSend) { |
|
|
|
private void timeoutCallsToSend(TimeoutProcessor processor, Map<Node, List<Call>> callsToSend) { |
|
|
|
int numTimedOut = 0; |
|
|
|
int numTimedOut = 0; |
|
|
|
for (List<Call> callList : callsToSend.values()) { |
|
|
|
for (List<Call> callList : callsToSend.values()) { |
|
|
|
numTimedOut += timeoutCalls(now, callList); |
|
|
|
numTimedOut += processor.handleTimeouts(callList, |
|
|
|
|
|
|
|
"Timed out waiting to send the call."); |
|
|
|
} |
|
|
|
} |
|
|
|
if (numTimedOut > 0) |
|
|
|
if (numTimedOut > 0) |
|
|
|
log.debug("{}: timed out {} call(s) with assigned nodes.", clientId, numTimedOut); |
|
|
|
log.debug("{}: timed out {} call(s) with assigned nodes.", clientId, numTimedOut); |
|
|
@ -698,10 +779,10 @@ public class KafkaAdminClient extends AdminClient { |
|
|
|
* even be in the process of being processed by the remote server. At the moment, our only option |
|
|
|
* even be in the process of being processed by the remote server. At the moment, our only option |
|
|
|
* to time them out is to close the entire connection. |
|
|
|
* to time them out is to close the entire connection. |
|
|
|
* |
|
|
|
* |
|
|
|
* @param now The current time in milliseconds. |
|
|
|
* @param processor The timeout processor. |
|
|
|
* @param callsInFlight A map of nodes to the calls they have in flight. |
|
|
|
* @param callsInFlight A map of nodes to the calls they have in flight. |
|
|
|
*/ |
|
|
|
*/ |
|
|
|
private void timeoutCallsInFlight(long now, Map<String, List<Call>> callsInFlight) { |
|
|
|
private void timeoutCallsInFlight(TimeoutProcessor processor, Map<String, List<Call>> callsInFlight) { |
|
|
|
int numTimedOut = 0; |
|
|
|
int numTimedOut = 0; |
|
|
|
for (Map.Entry<String, List<Call>> entry : callsInFlight.entrySet()) { |
|
|
|
for (Map.Entry<String, List<Call>> entry : callsInFlight.entrySet()) { |
|
|
|
List<Call> contexts = entry.getValue(); |
|
|
|
List<Call> contexts = entry.getValue(); |
|
|
@ -711,7 +792,7 @@ public class KafkaAdminClient extends AdminClient { |
|
|
|
// We assume that the first element in the list is the earliest. So it should be the
|
|
|
|
// We assume that the first element in the list is the earliest. So it should be the
|
|
|
|
// only one we need to check the timeout for.
|
|
|
|
// only one we need to check the timeout for.
|
|
|
|
Call call = contexts.get(0); |
|
|
|
Call call = contexts.get(0); |
|
|
|
if (calcTimeoutMsRemainingAsInt(now, call.deadlineMs) < 0) { |
|
|
|
if (processor.callHasExpired(call)) { |
|
|
|
log.debug("{}: Closing connection to {} to time out {}", clientId, nodeId, call); |
|
|
|
log.debug("{}: Closing connection to {} to time out {}", clientId, nodeId, call); |
|
|
|
client.close(nodeId); |
|
|
|
client.close(nodeId); |
|
|
|
numTimedOut++; |
|
|
|
numTimedOut++; |
|
|
@ -773,6 +854,22 @@ public class KafkaAdminClient extends AdminClient { |
|
|
|
} |
|
|
|
} |
|
|
|
} |
|
|
|
} |
|
|
|
|
|
|
|
|
|
|
|
|
|
|
|
private synchronized boolean threadShouldExit(long now, long curHardShutdownTimeMs, |
|
|
|
|
|
|
|
Map<Node, List<Call>> callsToSend, Map<Integer, Call> correlationIdToCalls) { |
|
|
|
|
|
|
|
if (newCalls.isEmpty() && callsToSend.isEmpty() && correlationIdToCalls.isEmpty()) { |
|
|
|
|
|
|
|
log.trace("{}: all work has been completed, and the I/O thread is now " + |
|
|
|
|
|
|
|
"exiting.", clientId); |
|
|
|
|
|
|
|
return true; |
|
|
|
|
|
|
|
} |
|
|
|
|
|
|
|
if (now > curHardShutdownTimeMs) { |
|
|
|
|
|
|
|
log.info("{}: forcing a hard I/O thread shutdown. Requests in progress will " + |
|
|
|
|
|
|
|
"be aborted.", clientId); |
|
|
|
|
|
|
|
return true; |
|
|
|
|
|
|
|
} |
|
|
|
|
|
|
|
log.debug("{}: hard shutdown in {} ms.", clientId, curHardShutdownTimeMs - now); |
|
|
|
|
|
|
|
return false; |
|
|
|
|
|
|
|
} |
|
|
|
|
|
|
|
|
|
|
|
@Override |
|
|
|
@Override |
|
|
|
public void run() { |
|
|
|
public void run() { |
|
|
|
/** |
|
|
|
/** |
|
|
@ -798,18 +895,25 @@ public class KafkaAdminClient extends AdminClient { |
|
|
|
long now = time.milliseconds(); |
|
|
|
long now = time.milliseconds(); |
|
|
|
log.trace("{} thread starting", clientId); |
|
|
|
log.trace("{} thread starting", clientId); |
|
|
|
while (true) { |
|
|
|
while (true) { |
|
|
|
// Check if the AdminClient is shutting down.
|
|
|
|
// Check if the AdminClient thread should shut down.
|
|
|
|
if (closed) |
|
|
|
long curHardShutdownTimeMs = hardShutdownTimeMs.get(); |
|
|
|
|
|
|
|
if ((curHardShutdownTimeMs != INVALID_SHUTDOWN_TIME) && |
|
|
|
|
|
|
|
threadShouldExit(now, curHardShutdownTimeMs, callsToSend, correlationIdToCalls)) |
|
|
|
break; |
|
|
|
break; |
|
|
|
|
|
|
|
|
|
|
|
// Handle timeouts.
|
|
|
|
// Handle timeouts.
|
|
|
|
timeoutNewCalls(now); |
|
|
|
TimeoutProcessor timeoutProcessor = new TimeoutProcessor(now); |
|
|
|
timeoutCallsToSend(now, callsToSend); |
|
|
|
timeoutNewCalls(timeoutProcessor); |
|
|
|
timeoutCallsInFlight(now, callsInFlight); |
|
|
|
timeoutCallsToSend(timeoutProcessor, callsToSend); |
|
|
|
|
|
|
|
timeoutCallsInFlight(timeoutProcessor, callsInFlight); |
|
|
|
|
|
|
|
|
|
|
|
|
|
|
|
long pollTimeout = Math.min(1200000, timeoutProcessor.nextTimeoutMs()); |
|
|
|
|
|
|
|
if (curHardShutdownTimeMs != INVALID_SHUTDOWN_TIME) { |
|
|
|
|
|
|
|
pollTimeout = Math.min(pollTimeout, curHardShutdownTimeMs - now); |
|
|
|
|
|
|
|
} |
|
|
|
|
|
|
|
|
|
|
|
// Handle new calls and metadata update requests.
|
|
|
|
// Handle new calls and metadata update requests.
|
|
|
|
prevMetadataVersion = checkMetadataReady(prevMetadataVersion); |
|
|
|
prevMetadataVersion = checkMetadataReady(prevMetadataVersion); |
|
|
|
long pollTimeout = 1200000; |
|
|
|
|
|
|
|
if (prevMetadataVersion == null) { |
|
|
|
if (prevMetadataVersion == null) { |
|
|
|
chooseNodesForNewCalls(now, callsToSend); |
|
|
|
chooseNodesForNewCalls(now, callsToSend); |
|
|
|
pollTimeout = Math.min(pollTimeout, |
|
|
|
pollTimeout = Math.min(pollTimeout, |
|
|
@ -826,10 +930,14 @@ public class KafkaAdminClient extends AdminClient { |
|
|
|
handleResponses(now, responses, callsInFlight, correlationIdToCalls); |
|
|
|
handleResponses(now, responses, callsInFlight, correlationIdToCalls); |
|
|
|
} |
|
|
|
} |
|
|
|
int numTimedOut = 0; |
|
|
|
int numTimedOut = 0; |
|
|
|
|
|
|
|
TimeoutProcessor timeoutProcessor = new TimeoutProcessor(Long.MAX_VALUE); |
|
|
|
synchronized (this) { |
|
|
|
synchronized (this) { |
|
|
|
numTimedOut += timeoutCalls(Long.MAX_VALUE, newCalls); |
|
|
|
numTimedOut += timeoutProcessor.handleTimeouts(newCalls, |
|
|
|
|
|
|
|
"The AdminClient thread has exited."); |
|
|
|
|
|
|
|
newCalls = null; |
|
|
|
} |
|
|
|
} |
|
|
|
numTimedOut += timeoutCalls(Long.MAX_VALUE, correlationIdToCalls.values()); |
|
|
|
numTimedOut += timeoutProcessor.handleTimeouts(correlationIdToCalls.values(), |
|
|
|
|
|
|
|
"The AdminClient thread has exited."); |
|
|
|
if (numTimedOut > 0) { |
|
|
|
if (numTimedOut > 0) { |
|
|
|
log.debug("{}: timed out {} remaining operations.", clientId, numTimedOut); |
|
|
|
log.debug("{}: timed out {} remaining operations.", clientId, numTimedOut); |
|
|
|
} |
|
|
|
} |
|
|
@ -838,15 +946,51 @@ public class KafkaAdminClient extends AdminClient { |
|
|
|
log.debug("{}: exiting AdminClientRunnable thread.", clientId); |
|
|
|
log.debug("{}: exiting AdminClientRunnable thread.", clientId); |
|
|
|
} |
|
|
|
} |
|
|
|
|
|
|
|
|
|
|
|
void call(Call call, long now) { |
|
|
|
/** |
|
|
|
|
|
|
|
* Queue a call for sending. |
|
|
|
|
|
|
|
* |
|
|
|
|
|
|
|
* If the AdminClient thread has exited, this will fail. Otherwise, it will succeed (even |
|
|
|
|
|
|
|
* if the AdminClient is shutting down.) This function should called when retrying an |
|
|
|
|
|
|
|
* existing call. |
|
|
|
|
|
|
|
* |
|
|
|
|
|
|
|
* @param call The new call object. |
|
|
|
|
|
|
|
* @param now The current time in milliseconds. |
|
|
|
|
|
|
|
*/ |
|
|
|
|
|
|
|
void enqueue(Call call, long now) { |
|
|
|
if (log.isDebugEnabled()) { |
|
|
|
if (log.isDebugEnabled()) { |
|
|
|
log.debug("{}: queueing {} with a timeout {} ms from now.", |
|
|
|
log.debug("{}: queueing {} with a timeout {} ms from now.", |
|
|
|
clientId, call, call.deadlineMs - now); |
|
|
|
clientId, call, call.deadlineMs - now); |
|
|
|
} |
|
|
|
} |
|
|
|
|
|
|
|
boolean accepted = false; |
|
|
|
synchronized (this) { |
|
|
|
synchronized (this) { |
|
|
|
newCalls.add(call); |
|
|
|
if (newCalls != null) { |
|
|
|
|
|
|
|
newCalls.add(call); |
|
|
|
|
|
|
|
accepted = true; |
|
|
|
|
|
|
|
} |
|
|
|
|
|
|
|
} |
|
|
|
|
|
|
|
if (accepted) { |
|
|
|
|
|
|
|
client.wakeup(); // wake the thread if it is in poll()
|
|
|
|
|
|
|
|
} else { |
|
|
|
|
|
|
|
log.debug("{}: the AdminClient thread has exited. Timing out {}.", clientId, call); |
|
|
|
|
|
|
|
call.fail(Long.MAX_VALUE, new TimeoutException("The AdminClient thread has exited.")); |
|
|
|
|
|
|
|
} |
|
|
|
|
|
|
|
} |
|
|
|
|
|
|
|
|
|
|
|
|
|
|
|
/** |
|
|
|
|
|
|
|
* Initiate a new call. |
|
|
|
|
|
|
|
* |
|
|
|
|
|
|
|
* This will fail if the AdminClient is scheduled to shut down. |
|
|
|
|
|
|
|
* |
|
|
|
|
|
|
|
* @param call The new call object. |
|
|
|
|
|
|
|
* @param now The current time in milliseconds. |
|
|
|
|
|
|
|
*/ |
|
|
|
|
|
|
|
void call(Call call, long now) { |
|
|
|
|
|
|
|
if (hardShutdownTimeMs.get() != INVALID_SHUTDOWN_TIME) { |
|
|
|
|
|
|
|
log.debug("{}: the AdminClient is not accepting new calls. Timing out {}.", clientId, call); |
|
|
|
|
|
|
|
call.fail(Long.MAX_VALUE, new TimeoutException("The AdminClient thread is not accepting new calls.")); |
|
|
|
|
|
|
|
} else { |
|
|
|
|
|
|
|
enqueue(call, now); |
|
|
|
} |
|
|
|
} |
|
|
|
client.wakeup(); |
|
|
|
|
|
|
|
} |
|
|
|
} |
|
|
|
} |
|
|
|
} |
|
|
|
|
|
|
|
|
|
|
|