unavailableTopics, long now) {
Objects.requireNonNull(newCluster, "cluster should not be null");
+ if (isClosed())
+ throw new IllegalStateException("Update requested after metadata close");
this.needUpdate = false;
this.lastRefreshMs = now;
@@ -331,6 +351,25 @@ public synchronized void removeListener(Listener listener) {
this.listeners.remove(listener);
}
+ /**
+ * "Close" this metadata instance to indicate that metadata updates are no longer possible. This is typically used
+ * when the thread responsible for performing metadata updates is exiting and needs a way to relay this information
+ * to any other thread(s) that could potentially wait on metadata update to come through.
+ */
+ @Override
+ public synchronized void close() {
+ this.isClosed = true;
+ this.notifyAll();
+ }
+
+ /**
+ * Check if this metadata instance has been closed. See {@link #close()} for more information.
+ * @return True if this instance has been closed; false otherwise
+ */
+ public synchronized boolean isClosed() {
+ return this.isClosed;
+ }
+
/**
* MetadataUpdate Listener
*/
diff --git a/clients/src/main/java/org/apache/kafka/clients/MetadataUpdater.java b/clients/src/main/java/org/apache/kafka/clients/MetadataUpdater.java
index 09ed995d14c3e..de765db5a8db3 100644
--- a/clients/src/main/java/org/apache/kafka/clients/MetadataUpdater.java
+++ b/clients/src/main/java/org/apache/kafka/clients/MetadataUpdater.java
@@ -21,6 +21,7 @@
import org.apache.kafka.common.requests.MetadataResponse;
import org.apache.kafka.common.requests.RequestHeader;
+import java.io.Closeable;
import java.util.List;
/**
@@ -29,7 +30,7 @@
*
* This class is not thread-safe!
*/
-public interface MetadataUpdater {
+public interface MetadataUpdater extends Closeable {
/**
* Gets the current cluster info without blocking.
@@ -82,4 +83,10 @@ public interface MetadataUpdater {
* start of the update if possible (see `maybeUpdate` for more information).
*/
void requestUpdate();
+
+ /**
+ * Close this updater.
+ */
+ @Override
+ void close();
}
diff --git a/clients/src/main/java/org/apache/kafka/clients/NetworkClient.java b/clients/src/main/java/org/apache/kafka/clients/NetworkClient.java
index 7c87277eb8f0a..4f24b54f76eb1 100644
--- a/clients/src/main/java/org/apache/kafka/clients/NetworkClient.java
+++ b/clients/src/main/java/org/apache/kafka/clients/NetworkClient.java
@@ -18,7 +18,9 @@
import org.apache.kafka.common.Cluster;
import org.apache.kafka.common.Node;
+import org.apache.kafka.common.TopicPartition;
import org.apache.kafka.common.errors.AuthenticationException;
+import org.apache.kafka.common.errors.DisconnectException;
import org.apache.kafka.common.errors.UnsupportedVersionException;
import org.apache.kafka.common.metrics.Sensor;
import org.apache.kafka.common.network.ChannelState;
@@ -52,6 +54,8 @@
import java.util.List;
import java.util.Map;
import java.util.Random;
+import java.util.concurrent.atomic.AtomicReference;
+import java.util.stream.Collectors;
/**
* A network client for asynchronous request/response network i/o. This is an internal class used to implement the
@@ -61,6 +65,12 @@
*/
public class NetworkClient implements KafkaClient {
+ private enum State {
+ ACTIVE,
+ CLOSING,
+ CLOSED
+ }
+
private final Logger log;
/* the selector used to perform network i/o */
@@ -88,8 +98,8 @@ public class NetworkClient implements KafkaClient {
/* the current correlation id to use when sending requests to servers */
private int correlation;
- /* max time in ms for the producer to wait for acknowledgement from server*/
- private final int requestTimeoutMs;
+ /* default timeout for individual requests to await acknowledgement from servers */
+ private final int defaultRequestTimeoutMs;
/* time in ms to wait before retrying to create connection to a server */
private final long reconnectBackoffMs;
@@ -109,6 +119,8 @@ public class NetworkClient implements KafkaClient {
private final Sensor throttleTimeSensor;
+ private final AtomicReference state;
+
public NetworkClient(Selectable selector,
Metadata metadata,
String clientId,
@@ -117,15 +129,26 @@ public NetworkClient(Selectable selector,
long reconnectBackoffMax,
int socketSendBuffer,
int socketReceiveBuffer,
- int requestTimeoutMs,
+ int defaultRequestTimeoutMs,
Time time,
boolean discoverBrokerVersions,
ApiVersions apiVersions,
LogContext logContext) {
- this(null, metadata, selector, clientId, maxInFlightRequestsPerConnection,
- reconnectBackoffMs, reconnectBackoffMax,
- socketSendBuffer, socketReceiveBuffer, requestTimeoutMs, time,
- discoverBrokerVersions, apiVersions, null, logContext);
+ this(null,
+ metadata,
+ selector,
+ clientId,
+ maxInFlightRequestsPerConnection,
+ reconnectBackoffMs,
+ reconnectBackoffMax,
+ socketSendBuffer,
+ socketReceiveBuffer,
+ defaultRequestTimeoutMs,
+ time,
+ discoverBrokerVersions,
+ apiVersions,
+ null,
+ logContext);
}
public NetworkClient(Selectable selector,
@@ -136,16 +159,27 @@ public NetworkClient(Selectable selector,
long reconnectBackoffMax,
int socketSendBuffer,
int socketReceiveBuffer,
- int requestTimeoutMs,
+ int defaultRequestTimeoutMs,
Time time,
boolean discoverBrokerVersions,
ApiVersions apiVersions,
Sensor throttleTimeSensor,
LogContext logContext) {
- this(null, metadata, selector, clientId, maxInFlightRequestsPerConnection,
- reconnectBackoffMs, reconnectBackoffMax,
- socketSendBuffer, socketReceiveBuffer, requestTimeoutMs, time,
- discoverBrokerVersions, apiVersions, throttleTimeSensor, logContext);
+ this(null,
+ metadata,
+ selector,
+ clientId,
+ maxInFlightRequestsPerConnection,
+ reconnectBackoffMs,
+ reconnectBackoffMax,
+ socketSendBuffer,
+ socketReceiveBuffer,
+ defaultRequestTimeoutMs,
+ time,
+ discoverBrokerVersions,
+ apiVersions,
+ throttleTimeSensor,
+ logContext);
}
public NetworkClient(Selectable selector,
@@ -156,15 +190,26 @@ public NetworkClient(Selectable selector,
long reconnectBackoffMax,
int socketSendBuffer,
int socketReceiveBuffer,
- int requestTimeoutMs,
+ int defaultRequestTimeoutMs,
Time time,
boolean discoverBrokerVersions,
ApiVersions apiVersions,
LogContext logContext) {
- this(metadataUpdater, null, selector, clientId, maxInFlightRequestsPerConnection,
- reconnectBackoffMs, reconnectBackoffMax,
- socketSendBuffer, socketReceiveBuffer, requestTimeoutMs, time,
- discoverBrokerVersions, apiVersions, null, logContext);
+ this(metadataUpdater,
+ null,
+ selector,
+ clientId,
+ maxInFlightRequestsPerConnection,
+ reconnectBackoffMs,
+ reconnectBackoffMax,
+ socketSendBuffer,
+ socketReceiveBuffer,
+ defaultRequestTimeoutMs,
+ time,
+ discoverBrokerVersions,
+ apiVersions,
+ null,
+ logContext);
}
private NetworkClient(MetadataUpdater metadataUpdater,
@@ -176,7 +221,7 @@ private NetworkClient(MetadataUpdater metadataUpdater,
long reconnectBackoffMax,
int socketSendBuffer,
int socketReceiveBuffer,
- int requestTimeoutMs,
+ int defaultRequestTimeoutMs,
Time time,
boolean discoverBrokerVersions,
ApiVersions apiVersions,
@@ -201,13 +246,14 @@ private NetworkClient(MetadataUpdater metadataUpdater,
this.socketReceiveBuffer = socketReceiveBuffer;
this.correlation = 0;
this.randOffset = new Random();
- this.requestTimeoutMs = requestTimeoutMs;
+ this.defaultRequestTimeoutMs = defaultRequestTimeoutMs;
this.reconnectBackoffMs = reconnectBackoffMs;
this.time = time;
this.discoverBrokerVersions = discoverBrokerVersions;
this.apiVersions = apiVersions;
this.throttleTimeSensor = throttleTimeSensor;
this.log = logContext.logger(NetworkClient.class);
+ this.state = new AtomicReference<>(State.ACTIVE);
}
/**
@@ -383,6 +429,7 @@ private void sendInternalMetadataRequest(MetadataRequest.Builder builder,
}
private void doSend(ClientRequest clientRequest, boolean isInternalRequest, long now) {
+ ensureActive();
String nodeId = clientRequest.destination();
if (!isInternalRequest) {
// If this request came from outside the NetworkClient, validate
@@ -426,31 +473,28 @@ private void doSend(ClientRequest clientRequest, boolean isInternalRequest, long
}
private void doSend(ClientRequest clientRequest, boolean isInternalRequest, long now, AbstractRequest request) {
- String nodeId = clientRequest.destination();
+ String destination = clientRequest.destination();
RequestHeader header = clientRequest.makeHeader(request.version());
if (log.isDebugEnabled()) {
int latestClientVersion = clientRequest.apiKey().latestVersion();
if (header.apiVersion() == latestClientVersion) {
log.trace("Sending {} {} with correlation id {} to node {}", clientRequest.apiKey(), request,
- clientRequest.correlationId(), nodeId);
+ clientRequest.correlationId(), destination);
} else {
log.debug("Using older server API v{} to send {} {} with correlation id {} to node {}",
- header.apiVersion(), clientRequest.apiKey(), request, clientRequest.correlationId(), nodeId);
+ header.apiVersion(), clientRequest.apiKey(), request, clientRequest.correlationId(), destination);
}
}
- Send send = request.toSend(nodeId, header);
+ Send send = request.toSend(destination, header);
InFlightRequest inFlightRequest = new InFlightRequest(
+ clientRequest,
header,
- clientRequest.createdTimeMs(),
- clientRequest.destination(),
- clientRequest.callback(),
- clientRequest.expectResponse(),
isInternalRequest,
request,
send,
now);
this.inFlightRequests.add(inFlightRequest);
- selector.send(inFlightRequest.send);
+ selector.send(send);
}
/**
@@ -464,6 +508,8 @@ private void doSend(ClientRequest clientRequest, boolean isInternalRequest, long
*/
@Override
public List poll(long timeout, long now) {
+ ensureActive();
+
if (!abortedSends.isEmpty()) {
// If there are aborted sends because of unsupported version exceptions or disconnects,
// handle them immediately without waiting for Selector#poll.
@@ -475,7 +521,7 @@ public List poll(long timeout, long now) {
long metadataTimeout = metadataUpdater.maybeUpdate(now);
try {
- this.selector.poll(Utils.min(timeout, metadataTimeout, requestTimeoutMs));
+ this.selector.poll(Utils.min(timeout, metadataTimeout, defaultRequestTimeoutMs));
} catch (IOException e) {
log.error("Unexpected error during I/O", e);
}
@@ -543,12 +589,35 @@ public void wakeup() {
this.selector.wakeup();
}
+ @Override
+ public void initiateClose() {
+ if (state.compareAndSet(State.ACTIVE, State.CLOSING)) {
+ wakeup();
+ }
+ }
+
+ @Override
+ public boolean active() {
+ return state.get() == State.ACTIVE;
+ }
+
+ private void ensureActive() {
+ if (!active())
+ throw new DisconnectException("NetworkClient is no longer active, state is " + state);
+ }
+
/**
* Close the network client
*/
@Override
public void close() {
- this.selector.close();
+ state.compareAndSet(State.ACTIVE, State.CLOSING);
+ if (state.compareAndSet(State.CLOSING, State.CLOSED)) {
+ this.selector.close();
+ this.metadataUpdater.close();
+ } else {
+ log.warn("Attempting to close NetworkClient that has already been closed.");
+ }
}
/**
@@ -658,7 +727,7 @@ else if (request.header.apiKey() == ApiKeys.METADATA)
* @param now The current time
*/
private void handleTimedOutRequests(List responses, long now) {
- List nodeIds = this.inFlightRequests.getNodesWithTimedOutRequests(now, this.requestTimeoutMs);
+ List nodeIds = this.inFlightRequests.nodesWithTimedOutRequests(now);
for (String nodeId : nodeIds) {
// close connection to the node
this.selector.close(nodeId);
@@ -837,7 +906,7 @@ private void initiateConnect(Node node, long now) {
connectionStates.disconnected(nodeConnectionId, now);
/* maybe the problem is our metadata, update it */
metadataUpdater.requestUpdate();
- log.debug("Error connecting to node {}", node, e);
+ log.warn("Error connecting to node {}", node, e);
}
}
@@ -868,7 +937,7 @@ public boolean isUpdateDue(long now) {
public long maybeUpdate(long now) {
// should we update our metadata?
long timeToNextMetadataUpdate = metadata.timeToNextUpdate(now);
- long waitForMetadataFetch = this.metadataFetchInProgress ? requestTimeoutMs : 0;
+ long waitForMetadataFetch = this.metadataFetchInProgress ? defaultRequestTimeoutMs : 0;
long metadataTimeout = Math.max(timeToNextMetadataUpdate, waitForMetadataFetch);
@@ -915,6 +984,20 @@ public void handleAuthenticationFailure(AuthenticationException exception) {
public void handleCompletedMetadataResponse(RequestHeader requestHeader, long now, MetadataResponse response) {
this.metadataFetchInProgress = false;
Cluster cluster = response.cluster();
+
+ // If any partition has leader with missing listeners, log a few for diagnosing broker configuration
+ // issues. This could be a transient issue if listeners were added dynamically to brokers.
+ List missingListenerPartitions = response.topicMetadata().stream().flatMap(topicMetadata ->
+ topicMetadata.partitionMetadata().stream()
+ .filter(partitionMetadata -> partitionMetadata.error() == Errors.LISTENER_NOT_FOUND)
+ .map(partitionMetadata -> new TopicPartition(topicMetadata.topic(), partitionMetadata.partition())))
+ .collect(Collectors.toList());
+ if (!missingListenerPartitions.isEmpty()) {
+ int count = missingListenerPartitions.size();
+ log.warn("{} partitions have leader brokers without a matching listener, including {}",
+ count, missingListenerPartitions.subList(0, Math.min(10, count)));
+ }
+
// check if any topics metadata failed to get updated
Map errors = response.errors();
if (!errors.isEmpty())
@@ -935,6 +1018,11 @@ public void requestUpdate() {
this.metadata.requestUpdate();
}
+ @Override
+ public void close() {
+ this.metadata.close();
+ }
+
/**
* Return true if there's at least one connection establishment is currently underway
*/
@@ -965,7 +1053,7 @@ private long maybeUpdate(long now, Node node) {
log.debug("Sending metadata request {} to node {}", metadataRequest, node);
sendInternalMetadataRequest(metadataRequest, nodeConnectionId, now);
- return requestTimeoutMs;
+ return defaultRequestTimeoutMs;
}
// If there's any connection establishment underway, wait until it completes. This prevents
@@ -993,16 +1081,26 @@ private long maybeUpdate(long now, Node node) {
}
@Override
- public ClientRequest newClientRequest(String nodeId, AbstractRequest.Builder> requestBuilder, long createdTimeMs,
+ public ClientRequest newClientRequest(String nodeId,
+ AbstractRequest.Builder> requestBuilder,
+ long createdTimeMs,
boolean expectResponse) {
- return newClientRequest(nodeId, requestBuilder, createdTimeMs, expectResponse, null);
+ return newClientRequest(nodeId, requestBuilder, createdTimeMs, expectResponse, defaultRequestTimeoutMs, null);
}
@Override
- public ClientRequest newClientRequest(String nodeId, AbstractRequest.Builder> requestBuilder, long createdTimeMs,
- boolean expectResponse, RequestCompletionHandler callback) {
+ public ClientRequest newClientRequest(String nodeId,
+ AbstractRequest.Builder> requestBuilder,
+ long createdTimeMs,
+ boolean expectResponse,
+ int requestTimeoutMs,
+ RequestCompletionHandler callback) {
return new ClientRequest(nodeId, requestBuilder, correlation++, clientId, createdTimeMs, expectResponse,
- callback);
+ requestTimeoutMs, callback);
+ }
+
+ public boolean discoverBrokerVersions() {
+ return discoverBrokerVersions;
}
static class InFlightRequest {
@@ -1015,8 +1113,28 @@ static class InFlightRequest {
final Send send;
final long sendTimeMs;
final long createdTimeMs;
+ final long requestTimeoutMs;
+
+ public InFlightRequest(ClientRequest clientRequest,
+ RequestHeader header,
+ boolean isInternalRequest,
+ AbstractRequest request,
+ Send send,
+ long sendTimeMs) {
+ this(header,
+ clientRequest.requestTimeoutMs(),
+ clientRequest.createdTimeMs(),
+ clientRequest.destination(),
+ clientRequest.callback(),
+ clientRequest.expectResponse(),
+ isInternalRequest,
+ request,
+ send,
+ sendTimeMs);
+ }
public InFlightRequest(RequestHeader header,
+ int requestTimeoutMs,
long createdTimeMs,
String destination,
RequestCompletionHandler callback,
@@ -1026,6 +1144,8 @@ public InFlightRequest(RequestHeader header,
Send send,
long sendTimeMs) {
this.header = header;
+ this.requestTimeoutMs = requestTimeoutMs;
+ this.createdTimeMs = createdTimeMs;
this.destination = destination;
this.callback = callback;
this.expectResponse = expectResponse;
@@ -1033,7 +1153,6 @@ public InFlightRequest(RequestHeader header,
this.request = request;
this.send = send;
this.sendTimeMs = sendTimeMs;
- this.createdTimeMs = createdTimeMs;
}
public ClientResponse completed(AbstractResponse response, long timeMs) {
@@ -1060,7 +1179,4 @@ public String toString() {
}
}
- public boolean discoverBrokerVersions() {
- return discoverBrokerVersions;
- }
}
diff --git a/clients/src/main/java/org/apache/kafka/clients/NetworkClientUtils.java b/clients/src/main/java/org/apache/kafka/clients/NetworkClientUtils.java
index 94fe288090ec3..c952b82462def 100644
--- a/clients/src/main/java/org/apache/kafka/clients/NetworkClientUtils.java
+++ b/clients/src/main/java/org/apache/kafka/clients/NetworkClientUtils.java
@@ -18,6 +18,7 @@
package org.apache.kafka.clients;
import org.apache.kafka.common.Node;
+import org.apache.kafka.common.errors.DisconnectException;
import org.apache.kafka.common.utils.Time;
import java.io.IOException;
@@ -83,25 +84,35 @@ public static boolean awaitReady(KafkaClient client, Node node, Time time, long
* disconnection happens (which can happen for a number of reasons including a request timeout).
*
* In case of a disconnection, an `IOException` is thrown.
+ * If shutdown is initiated on the client during this method, an IOException is thrown.
*
* This method is useful for implementing blocking behaviour on top of the non-blocking `NetworkClient`, use it with
* care.
*/
public static ClientResponse sendAndReceive(KafkaClient client, ClientRequest request, Time time) throws IOException {
- client.send(request, time.milliseconds());
- while (true) {
- List responses = client.poll(Long.MAX_VALUE, time.milliseconds());
- for (ClientResponse response : responses) {
- if (response.requestHeader().correlationId() == request.correlationId()) {
- if (response.wasDisconnected()) {
- throw new IOException("Connection to " + response.destination() + " was disconnected before the response was read");
+ try {
+ client.send(request, time.milliseconds());
+ while (client.active()) {
+ List responses = client.poll(Long.MAX_VALUE, time.milliseconds());
+ for (ClientResponse response : responses) {
+ if (response.requestHeader().correlationId() == request.correlationId()) {
+ if (response.wasDisconnected()) {
+ throw new IOException("Connection to " + response.destination() + " was disconnected before the response was read");
+ }
+ if (response.versionMismatch() != null) {
+ throw response.versionMismatch();
+ }
+ return response;
}
- if (response.versionMismatch() != null) {
- throw response.versionMismatch();
- }
- return response;
}
}
+ throw new IOException("Client was shutdown before response was read");
+ } catch (DisconnectException e) {
+ if (client.active())
+ throw e;
+ else
+ throw new IOException("Client was shutdown before response was read");
+
}
}
}
diff --git a/clients/src/main/java/org/apache/kafka/clients/admin/KafkaAdminClient.java b/clients/src/main/java/org/apache/kafka/clients/admin/KafkaAdminClient.java
index 450de06fcd104..5383536390e4a 100644
--- a/clients/src/main/java/org/apache/kafka/clients/admin/KafkaAdminClient.java
+++ b/clients/src/main/java/org/apache/kafka/clients/admin/KafkaAdminClient.java
@@ -112,8 +112,6 @@
import org.apache.kafka.common.requests.OffsetFetchResponse;
import org.apache.kafka.common.requests.RenewDelegationTokenRequest;
import org.apache.kafka.common.requests.RenewDelegationTokenResponse;
-import org.apache.kafka.common.requests.Resource;
-import org.apache.kafka.common.requests.ResourceType;
import org.apache.kafka.common.security.token.delegation.DelegationToken;
import org.apache.kafka.common.security.token.delegation.TokenInformation;
import org.apache.kafka.common.utils.AppInfoParser;
@@ -166,6 +164,11 @@ public class KafkaAdminClient extends AdminClient {
*/
private static final long INVALID_SHUTDOWN_TIME = -1;
+ /**
+ * Thread name prefix for admin client network thread
+ */
+ static final String NETWORK_THREAD_PREFIX = "kafka-admin-client-thread";
+
private final Logger log;
/**
@@ -409,7 +412,7 @@ private KafkaAdminClient(AdminClientConfig config,
this.metrics = metrics;
this.client = client;
this.runnable = new AdminClientRunnable();
- String threadName = "kafka-admin-client-thread | " + clientId;
+ String threadName = NETWORK_THREAD_PREFIX + " | " + clientId;
this.thread = new KafkaThread(threadName, runnable, true);
this.timeoutProcessorFactory = (timeoutProcessorFactory == null) ?
new TimeoutProcessorFactory() : timeoutProcessorFactory;
@@ -1683,19 +1686,19 @@ public DescribeConfigsResult describeConfigs(Collection configRe
// The BROKER resources which we want to describe. We must make a separate DescribeConfigs
// request for every BROKER resource we want to describe.
- final Collection brokerResources = new ArrayList<>();
+ final Collection brokerResources = new ArrayList<>();
// The non-BROKER resources which we want to describe. These resources can be described by a
// single, unified DescribeConfigs request.
- final Collection unifiedRequestResources = new ArrayList<>(configResources.size());
+ final Collection unifiedRequestResources = new ArrayList<>(configResources.size());
for (ConfigResource resource : configResources) {
if (resource.type() == ConfigResource.Type.BROKER && !resource.isDefault()) {
- brokerFutures.put(resource, new KafkaFutureImpl());
- brokerResources.add(configResourceToResource(resource));
+ brokerFutures.put(resource, new KafkaFutureImpl<>());
+ brokerResources.add(resource);
} else {
- unifiedRequestFutures.put(resource, new KafkaFutureImpl());
- unifiedRequestResources.add(configResourceToResource(resource));
+ unifiedRequestFutures.put(resource, new KafkaFutureImpl<>());
+ unifiedRequestResources.add(resource);
}
}
@@ -1716,7 +1719,7 @@ void handleResponse(AbstractResponse abstractResponse) {
for (Map.Entry> entry : unifiedRequestFutures.entrySet()) {
ConfigResource configResource = entry.getKey();
KafkaFutureImpl future = entry.getValue();
- DescribeConfigsResponse.Config config = response.config(configResourceToResource(configResource));
+ DescribeConfigsResponse.Config config = response.config(configResource);
if (config == null) {
future.completeExceptionally(new UnknownServerException(
"Malformed broker response: missing config for " + configResource));
@@ -1746,7 +1749,7 @@ void handleFailure(Throwable throwable) {
for (Map.Entry> entry : brokerFutures.entrySet()) {
final KafkaFutureImpl brokerFuture = entry.getValue();
- final Resource resource = configResourceToResource(entry.getKey());
+ final ConfigResource resource = entry.getKey();
final int nodeId = Integer.parseInt(resource.name());
runnable.call(new Call("describeBrokerConfigs", calcDeadlineMs(now, options.timeoutMs()),
new ConstantNodeIdProvider(nodeId)) {
@@ -1792,21 +1795,6 @@ void handleFailure(Throwable throwable) {
return new DescribeConfigsResult(allFutures);
}
- private Resource configResourceToResource(ConfigResource configResource) {
- ResourceType resourceType;
- switch (configResource.type()) {
- case TOPIC:
- resourceType = ResourceType.TOPIC;
- break;
- case BROKER:
- resourceType = ResourceType.BROKER;
- break;
- default:
- throw new IllegalArgumentException("Unexpected resource type " + configResource.type());
- }
- return new Resource(resourceType, configResource.name());
- }
-
private List configSynonyms(DescribeConfigsResponse.ConfigEntry configEntry) {
List synonyms = new ArrayList<>(configEntry.synonyms().size());
for (DescribeConfigsResponse.ConfigSynonym synonym : configEntry.synonyms()) {
@@ -1856,7 +1844,7 @@ public AlterConfigsResult alterConfigs(Map configs, fina
}
if (!unifiedRequestResources.isEmpty())
allFutures.putAll(alterConfigs(configs, options, unifiedRequestResources, new LeastLoadedNodeProvider()));
- return new AlterConfigsResult(new HashMap>(allFutures));
+ return new AlterConfigsResult(new HashMap<>(allFutures));
}
private Map> alterConfigs(Map configs,
@@ -1864,13 +1852,13 @@ private Map> alterConfigs(Map resources,
NodeProvider nodeProvider) {
final Map> futures = new HashMap<>();
- final Map requestMap = new HashMap<>(resources.size());
+ final Map requestMap = new HashMap<>(resources.size());
for (ConfigResource resource : resources) {
List configEntries = new ArrayList<>();
for (ConfigEntry configEntry: configs.get(resource).entries())
configEntries.add(new AlterConfigsRequest.ConfigEntry(configEntry.name(), configEntry.value()));
- requestMap.put(configResourceToResource(resource), new AlterConfigsRequest.Config(configEntries));
- futures.put(resource, new KafkaFutureImpl());
+ requestMap.put(resource, new AlterConfigsRequest.Config(configEntries));
+ futures.put(resource, new KafkaFutureImpl<>());
}
final long now = time.milliseconds();
@@ -1886,7 +1874,7 @@ public void handleResponse(AbstractResponse abstractResponse) {
AlterConfigsResponse response = (AlterConfigsResponse) abstractResponse;
for (Map.Entry> entry : futures.entrySet()) {
KafkaFutureImpl future = entry.getValue();
- ApiException exception = response.errors().get(configResourceToResource(entry.getKey())).exception();
+ ApiException exception = response.errors().get(entry.getKey()).exception();
if (exception != null) {
future.completeExceptionally(exception);
} else {
@@ -2407,16 +2395,9 @@ AbstractRequest.Builder createRequest(int timeoutMs) {
@Override
void handleResponse(AbstractResponse abstractResponse) {
final FindCoordinatorResponse fcResponse = (FindCoordinatorResponse) abstractResponse;
- Errors error = fcResponse.error();
- if (error == Errors.COORDINATOR_NOT_AVAILABLE) {
- // Retry COORDINATOR_NOT_AVAILABLE, in case the error is temporary.
- throw error.exception();
- } else if (error != Errors.NONE) {
- // All other errors are immediate failures.
- KafkaFutureImpl future = futures.get(groupId);
- future.completeExceptionally(error.exception());
+
+ if (handleGroupRequestError(fcResponse.error(), futures.get(groupId)))
return;
- }
final long nowDescribeConsumerGroups = time.milliseconds();
final int nodeId = fcResponse.node().id();
@@ -2433,39 +2414,36 @@ void handleResponse(AbstractResponse abstractResponse) {
KafkaFutureImpl future = futures.get(groupId);
final DescribeGroupsResponse.GroupMetadata groupMetadata = response.groups().get(groupId);
- final Errors groupError = groupMetadata.error();
- if (groupError != Errors.NONE) {
- // TODO: KAFKA-6789, we can retry based on the error code
- future.completeExceptionally(groupError.exception());
- } else {
- final String protocolType = groupMetadata.protocolType();
- if (protocolType.equals(ConsumerProtocol.PROTOCOL_TYPE) || protocolType.isEmpty()) {
- final List members = groupMetadata.members();
- final List memberDescriptions = new ArrayList<>(members.size());
-
- for (DescribeGroupsResponse.GroupMember groupMember : members) {
- Set partitions = Collections.emptySet();
- if (groupMember.memberAssignment().remaining() > 0) {
- final PartitionAssignor.Assignment assignment = ConsumerProtocol.
- deserializeAssignment(groupMember.memberAssignment().duplicate());
- partitions = new HashSet<>(assignment.partitions());
- }
- final MemberDescription memberDescription =
- new MemberDescription(groupMember.memberId(),
- groupMember.clientId(),
- groupMember.clientHost(),
- new MemberAssignment(partitions));
- memberDescriptions.add(memberDescription);
+ if (handleGroupRequestError(groupMetadata.error(), future))
+ return;
+
+ final String protocolType = groupMetadata.protocolType();
+ if (protocolType.equals(ConsumerProtocol.PROTOCOL_TYPE) || protocolType.isEmpty()) {
+ final List members = groupMetadata.members();
+ final List memberDescriptions = new ArrayList<>(members.size());
+
+ for (DescribeGroupsResponse.GroupMember groupMember : members) {
+ Set partitions = Collections.emptySet();
+ if (groupMember.memberAssignment().remaining() > 0) {
+ final PartitionAssignor.Assignment assignment = ConsumerProtocol.
+ deserializeAssignment(groupMember.memberAssignment().duplicate());
+ partitions = new HashSet<>(assignment.partitions());
}
- final ConsumerGroupDescription consumerGroupDescription =
- new ConsumerGroupDescription(groupId,
- protocolType.isEmpty(),
- memberDescriptions,
- groupMetadata.protocol(),
- ConsumerGroupState.parse(groupMetadata.state()),
- fcResponse.node());
- future.complete(consumerGroupDescription);
+ final MemberDescription memberDescription =
+ new MemberDescription(groupMember.memberId(),
+ groupMember.clientId(),
+ groupMember.clientHost(),
+ new MemberAssignment(partitions));
+ memberDescriptions.add(memberDescription);
}
+ final ConsumerGroupDescription consumerGroupDescription =
+ new ConsumerGroupDescription(groupId,
+ protocolType.isEmpty(),
+ memberDescriptions,
+ groupMetadata.protocol(),
+ ConsumerGroupState.parse(groupMetadata.state()),
+ fcResponse.node());
+ future.complete(consumerGroupDescription);
}
}
@@ -2488,6 +2466,17 @@ void handleFailure(Throwable throwable) {
return new DescribeConsumerGroupsResult(new HashMap>(futures));
}
+
+ private boolean handleGroupRequestError(Errors error, KafkaFutureImpl> future) {
+ if (error == Errors.COORDINATOR_LOAD_IN_PROGRESS || error == Errors.COORDINATOR_NOT_AVAILABLE) {
+ throw error.exception();
+ } else if (error != Errors.NONE) {
+ future.completeExceptionally(error.exception());
+ return true;
+ }
+ return false;
+ }
+
private final static class ListConsumerGroupsResults {
private final List errors;
private final HashMap listings;
@@ -2573,8 +2562,11 @@ private void maybeAddConsumerGroup(ListGroupsResponse.Group group) {
void handleResponse(AbstractResponse abstractResponse) {
final ListGroupsResponse response = (ListGroupsResponse) abstractResponse;
synchronized (results) {
- if (response.error() != Errors.NONE) {
- results.addError(response.error().exception(), node);
+ Errors error = response.error();
+ if (error == Errors.COORDINATOR_LOAD_IN_PROGRESS || error == Errors.COORDINATOR_NOT_AVAILABLE) {
+ throw error.exception();
+ } else if (error != Errors.NONE) {
+ results.addError(error.exception(), node);
} else {
for (ListGroupsResponse.Group group : response.groups()) {
maybeAddConsumerGroup(group);
@@ -2622,6 +2614,9 @@ AbstractRequest.Builder createRequest(int timeoutMs) {
void handleResponse(AbstractResponse abstractResponse) {
final FindCoordinatorResponse response = (FindCoordinatorResponse) abstractResponse;
+ if (handleGroupRequestError(response.error(), groupOffsetListingFuture))
+ return;
+
final long nowListConsumerGroupOffsets = time.milliseconds();
final int nodeId = response.node().id();
@@ -2637,24 +2632,23 @@ void handleResponse(AbstractResponse abstractResponse) {
final OffsetFetchResponse response = (OffsetFetchResponse) abstractResponse;
final Map groupOffsetsListing = new HashMap<>();
- if (response.hasError()) {
- groupOffsetListingFuture.completeExceptionally(response.error().exception());
- } else {
- for (Map.Entry entry :
- response.responseData().entrySet()) {
- final TopicPartition topicPartition = entry.getKey();
- final Errors error = entry.getValue().error;
-
- if (error == Errors.NONE) {
- final Long offset = entry.getValue().offset;
- final String metadata = entry.getValue().metadata;
- groupOffsetsListing.put(topicPartition, new OffsetAndMetadata(offset, metadata));
- } else {
- log.warn("Skipping return offset for {} due to error {}.", topicPartition, error);
- }
+ if (handleGroupRequestError(response.error(), groupOffsetListingFuture))
+ return;
+
+ for (Map.Entry entry :
+ response.responseData().entrySet()) {
+ final TopicPartition topicPartition = entry.getKey();
+ final Errors error = entry.getValue().error;
+
+ if (error == Errors.NONE) {
+ final Long offset = entry.getValue().offset;
+ final String metadata = entry.getValue().metadata;
+ groupOffsetsListing.put(topicPartition, new OffsetAndMetadata(offset, metadata));
+ } else {
+ log.warn("Skipping return offset for {} due to error {}.", topicPartition, error);
}
- groupOffsetListingFuture.complete(groupOffsetsListing);
}
+ groupOffsetListingFuture.complete(groupOffsetsListing);
}
@Override
@@ -2708,6 +2702,9 @@ AbstractRequest.Builder createRequest(int timeoutMs) {
void handleResponse(AbstractResponse abstractResponse) {
final FindCoordinatorResponse response = (FindCoordinatorResponse) abstractResponse;
+ if (handleGroupRequestError(response.error(), futures.get(groupId)))
+ return;
+
final long nowDeleteConsumerGroups = time.milliseconds();
final int nodeId = response.node().id();
@@ -2726,11 +2723,10 @@ void handleResponse(AbstractResponse abstractResponse) {
KafkaFutureImpl future = futures.get(groupId);
final Errors groupError = response.get(groupId);
- if (groupError != Errors.NONE) {
- future.completeExceptionally(groupError.exception());
- } else {
- future.complete(null);
- }
+ if (handleGroupRequestError(groupError, future))
+ return;
+
+ future.complete(null);
}
@Override
diff --git a/clients/src/main/java/org/apache/kafka/clients/admin/internals/AdminMetadataManager.java b/clients/src/main/java/org/apache/kafka/clients/admin/internals/AdminMetadataManager.java
index 85d3c28e8df51..1ad3991ca24d2 100644
--- a/clients/src/main/java/org/apache/kafka/clients/admin/internals/AdminMetadataManager.java
+++ b/clients/src/main/java/org/apache/kafka/clients/admin/internals/AdminMetadataManager.java
@@ -118,6 +118,10 @@ public void handleCompletedMetadataResponse(RequestHeader requestHeader, long no
public void requestUpdate() {
AdminMetadataManager.this.requestUpdate();
}
+
+ @Override
+ public void close() {
+ }
}
/**
diff --git a/clients/src/main/java/org/apache/kafka/clients/consumer/ConsumerConfig.java b/clients/src/main/java/org/apache/kafka/clients/consumer/ConsumerConfig.java
index 72e496cbd4697..fefeae343e00e 100644
--- a/clients/src/main/java/org/apache/kafka/clients/consumer/ConsumerConfig.java
+++ b/clients/src/main/java/org/apache/kafka/clients/consumer/ConsumerConfig.java
@@ -218,6 +218,10 @@ public class ConsumerConfig extends AbstractConfig {
public static final String REQUEST_TIMEOUT_MS_CONFIG = CommonClientConfigs.REQUEST_TIMEOUT_MS_CONFIG;
private static final String REQUEST_TIMEOUT_MS_DOC = CommonClientConfigs.REQUEST_TIMEOUT_MS_DOC;
+ /** default.api.timeout.ms */
+ public static final String DEFAULT_API_TIMEOUT_MS_CONFIG = "default.api.timeout.ms";
+ public static final String DEFAULT_API_TIMEOUT_MS_DOC = "Specifies the timeout (in milliseconds) for consumer APIs that could block. This configuration is used as the default timeout for all consumer operations that do not explicitly accept a timeout parameter.";
+
/** interceptor.classes */
public static final String INTERCEPTOR_CLASSES_CONFIG = "interceptor.classes";
public static final String INTERCEPTOR_CLASSES_DOC = "A list of classes to use as interceptors. "
@@ -399,10 +403,16 @@ public class ConsumerConfig extends AbstractConfig {
VALUE_DESERIALIZER_CLASS_DOC)
.define(REQUEST_TIMEOUT_MS_CONFIG,
Type.INT,
- 305000, // chosen to be higher than the default of max.poll.interval.ms
+ 30000,
atLeast(0),
Importance.MEDIUM,
REQUEST_TIMEOUT_MS_DOC)
+ .define(DEFAULT_API_TIMEOUT_MS_CONFIG,
+ Type.INT,
+ 60 * 1000,
+ atLeast(0),
+ Importance.MEDIUM,
+ DEFAULT_API_TIMEOUT_MS_DOC)
/* default is set to be a bit lower than the server default (10 min), to avoid both client and server closing connection at same time */
.define(CONNECTIONS_MAX_IDLE_MS_CONFIG,
Type.LONG,
diff --git a/clients/src/main/java/org/apache/kafka/clients/consumer/KafkaConsumer.java b/clients/src/main/java/org/apache/kafka/clients/consumer/KafkaConsumer.java
index 5bd6b935b3972..0818c9469c68c 100644
--- a/clients/src/main/java/org/apache/kafka/clients/consumer/KafkaConsumer.java
+++ b/clients/src/main/java/org/apache/kafka/clients/consumer/KafkaConsumer.java
@@ -25,6 +25,7 @@
import org.apache.kafka.clients.consumer.internals.ConsumerMetrics;
import org.apache.kafka.clients.consumer.internals.ConsumerNetworkClient;
import org.apache.kafka.clients.consumer.internals.Fetcher;
+import org.apache.kafka.clients.consumer.internals.Heartbeat;
import org.apache.kafka.clients.consumer.internals.NoOpConsumerRebalanceListener;
import org.apache.kafka.clients.consumer.internals.PartitionAssignor;
import org.apache.kafka.clients.consumer.internals.SubscriptionState;
@@ -34,7 +35,6 @@
import org.apache.kafka.common.MetricName;
import org.apache.kafka.common.PartitionInfo;
import org.apache.kafka.common.TopicPartition;
-import org.apache.kafka.common.config.ConfigException;
import org.apache.kafka.common.errors.InterruptException;
import org.apache.kafka.common.errors.TimeoutException;
import org.apache.kafka.common.internals.ClusterResourceListeners;
@@ -567,6 +567,7 @@ public class KafkaConsumer implements Consumer {
private final Metadata metadata;
private final long retryBackoffMs;
private final long requestTimeoutMs;
+ private final int defaultApiTimeoutMs;
private volatile boolean closed = false;
private List assignors;
@@ -666,10 +667,7 @@ private KafkaConsumer(ConsumerConfig config,
log.debug("Initializing the Kafka consumer");
this.requestTimeoutMs = config.getInt(ConsumerConfig.REQUEST_TIMEOUT_MS_CONFIG);
- int sessionTimeOutMs = config.getInt(ConsumerConfig.SESSION_TIMEOUT_MS_CONFIG);
- int fetchMaxWaitMs = config.getInt(ConsumerConfig.FETCH_MAX_WAIT_MS_CONFIG);
- if (this.requestTimeoutMs <= sessionTimeOutMs || this.requestTimeoutMs <= fetchMaxWaitMs)
- throw new ConfigException(ConsumerConfig.REQUEST_TIMEOUT_MS_CONFIG + " should be greater than " + ConsumerConfig.SESSION_TIMEOUT_MS_CONFIG + " and " + ConsumerConfig.FETCH_MAX_WAIT_MS_CONFIG);
+ this.defaultApiTimeoutMs = config.getInt(ConsumerConfig.DEFAULT_API_TIMEOUT_MS_CONFIG);
this.time = Time.SYSTEM;
Map metricsTags = Collections.singletonMap("client-id", clientId);
@@ -748,12 +746,15 @@ private KafkaConsumer(ConsumerConfig config,
this.assignors = config.getConfiguredInstances(
ConsumerConfig.PARTITION_ASSIGNMENT_STRATEGY_CONFIG,
PartitionAssignor.class);
+
+ int maxPollIntervalMs = config.getInt(ConsumerConfig.MAX_POLL_INTERVAL_MS_CONFIG);
+ int sessionTimeoutMs = config.getInt(ConsumerConfig.SESSION_TIMEOUT_MS_CONFIG);
this.coordinator = new ConsumerCoordinator(logContext,
this.client,
groupId,
- config.getInt(ConsumerConfig.MAX_POLL_INTERVAL_MS_CONFIG),
- config.getInt(ConsumerConfig.SESSION_TIMEOUT_MS_CONFIG),
- heartbeatIntervalMs,
+ maxPollIntervalMs,
+ sessionTimeoutMs,
+ new Heartbeat(sessionTimeoutMs, heartbeatIntervalMs, maxPollIntervalMs, retryBackoffMs),
assignors,
this.metadata,
this.subscriptions,
@@ -814,6 +815,7 @@ private KafkaConsumer(ConsumerConfig config,
Metadata metadata,
long retryBackoffMs,
long requestTimeoutMs,
+ int defaultApiTimeoutMs,
List assignors) {
this.log = logContext.logger(getClass());
this.clientId = clientId;
@@ -829,6 +831,7 @@ private KafkaConsumer(ConsumerConfig config,
this.metadata = metadata;
this.retryBackoffMs = retryBackoffMs;
this.requestTimeoutMs = requestTimeoutMs;
+ this.defaultApiTimeoutMs = defaultApiTimeoutMs;
this.assignors = assignors;
}
@@ -1268,8 +1271,9 @@ private long remainingTimeAtLeastZero(final long timeoutMs, final long elapsedTi
* every rebalance and also on startup. As such, if you need to store offsets in anything other than Kafka, this API
* should not be used.
*
- * This is a synchronous commits and will block until either the commit succeeds or an unrecoverable error is
- * encountered (in which case it is thrown to the caller).
+ * This is a synchronous commit and will block until either the commit succeeds, an unrecoverable error is
+ * encountered (in which case it is thrown to the caller), or the timeout specified by {@code default.api.timeout.ms} expires
+ * (in which case a {@link org.apache.kafka.common.errors.TimeoutException} is thrown to the caller).
*
* Note that asynchronous offset commits sent previously with the {@link #commitAsync(OffsetCommitCallback)}
* (or similar) are guaranteed to have their callbacks invoked prior to completion of this method.
@@ -1286,10 +1290,12 @@ private long remainingTimeAtLeastZero(final long timeoutMs, final long elapsedTi
* configured groupId. See the exception for more details
* @throws org.apache.kafka.common.KafkaException for any other unrecoverable errors (e.g. if offset metadata
* is too large or if the topic does not exist).
+ * @throws org.apache.kafka.common.errors.TimeoutException if the timeout specified by {@code default.api.timeout.ms} expires
+ * before successful completion of the offset commit
*/
@Override
public void commitSync() {
- commitSync(Duration.ofMillis(Long.MAX_VALUE));
+ commitSync(Duration.ofMillis(defaultApiTimeoutMs));
}
/**
@@ -1343,7 +1349,8 @@ public void commitSync(Duration timeout) {
* i.e. lastProcessedMessageOffset + 1.
*
* This is a synchronous commits and will block until either the commit succeeds or an unrecoverable error is
- * encountered (in which case it is thrown to the caller).
+ * encountered (in which case it is thrown to the caller), or the timeout specified by {@code default.api.timeout.ms} expires
+ * (in which case a {@link org.apache.kafka.common.errors.TimeoutException} is thrown to the caller).
*
* Note that asynchronous offset commits sent previously with the {@link #commitAsync(OffsetCommitCallback)}
* (or similar) are guaranteed to have their callbacks invoked prior to completion of this method.
@@ -1362,10 +1369,12 @@ public void commitSync(Duration timeout) {
* @throws java.lang.IllegalArgumentException if the committed offset is negative
* @throws org.apache.kafka.common.KafkaException for any other unrecoverable errors (e.g. if offset metadata
* is too large or if the topic does not exist).
+ * @throws org.apache.kafka.common.errors.TimeoutException if the timeout expires before successful completion
+ * of the offset commit
*/
@Override
public void commitSync(final Map offsets) {
- commitSync(offsets, Duration.ofMillis(Long.MAX_VALUE));
+ commitSync(offsets, Duration.ofMillis(defaultApiTimeoutMs));
}
/**
@@ -1560,7 +1569,8 @@ public void seekToEnd(Collection partitions) {
* This method may issue a remote call to the server if there is no current position for the given partition.
*
* This call will block until either the position could be determined or an unrecoverable error is
- * encountered (in which case it is thrown to the caller).
+ * encountered (in which case it is thrown to the caller), or the timeout specified by {@code default.api.timeout.ms} expires
+ * (in which case a {@link org.apache.kafka.common.errors.TimeoutException} is thrown to the caller).
*
* @param partition The partition to get the position for
* @return The current position of the consumer (that is, the offset of the next record to be fetched)
@@ -1575,10 +1585,12 @@ public void seekToEnd(Collection partitions) {
* @throws org.apache.kafka.common.errors.AuthorizationException if not authorized to the topic or to the
* configured groupId. See the exception for more details
* @throws org.apache.kafka.common.KafkaException for any other unrecoverable errors
+ * @throws org.apache.kafka.common.errors.TimeoutException if the position cannot be determined before the
+ * timeout specified by {@code default.api.timeout.ms} expires
*/
@Override
public long position(TopicPartition partition) {
- return position(partition, Duration.ofMillis(Long.MAX_VALUE));
+ return position(partition, Duration.ofMillis(defaultApiTimeoutMs));
}
/**
@@ -1592,7 +1604,7 @@ public long position(TopicPartition partition) {
* @param partition The partition to get the position for
* @param timeout The maximum amount of time to await determination of the current position
* @return The current position of the consumer (that is, the offset of the next record to be fetched)
- * @throws IllegalArgumentException if the provided TopicPartition is not assigned to this consumer
+ * @throws IllegalStateException if the provided TopicPartition is not assigned to this consumer
* @throws org.apache.kafka.clients.consumer.InvalidOffsetException if no offset is currently defined for
* the partition
* @throws org.apache.kafka.common.errors.WakeupException if {@link #wakeup()} is called before or while this
@@ -1641,7 +1653,10 @@ public long position(TopicPartition partition, final Duration timeout) {
* Get the last committed offset for the given partition (whether the commit happened by this process or
* another). This offset will be used as the position for the consumer in the event of a failure.
*
- * This call will block to do a remote call to get the latest committed offsets from the server.
+ * This call will do a remote call to get the latest committed offset from the server, and will block until the
+ * committed offset is gotten successfully, an unrecoverable error is encountered (in which case it is thrown to
+ * the caller), or the timeout specified by {@code default.api.timeout.ms} expires (in which case a
+ * {@link org.apache.kafka.common.errors.TimeoutException} is thrown to the caller).
*
* @param partition The partition to check
* @return The last committed offset and metadata or null if there was no prior commit
@@ -1653,10 +1668,12 @@ public long position(TopicPartition partition, final Duration timeout) {
* @throws org.apache.kafka.common.errors.AuthorizationException if not authorized to the topic or to the
* configured groupId. See the exception for more details
* @throws org.apache.kafka.common.KafkaException for any other unrecoverable errors
+ * @throws org.apache.kafka.common.errors.TimeoutException if the committed offset cannot be found before
+ * the timeout specified by {@code default.api.timeout.ms} expires.
*/
@Override
public OffsetAndMetadata committed(TopicPartition partition) {
- return committed(partition, Duration.ofMillis(Long.MAX_VALUE));
+ return committed(partition, Duration.ofMillis(defaultApiTimeoutMs));
}
/**
@@ -1718,11 +1735,11 @@ public OffsetAndMetadata committed(TopicPartition partition, final Duration time
* @throws org.apache.kafka.common.errors.AuthorizationException if not authorized to the specified topic. See the exception for more details
* @throws org.apache.kafka.common.KafkaException for any other unrecoverable errors
* @throws org.apache.kafka.common.errors.TimeoutException if the offset metadata could not be fetched before
- * the amount of time allocated by {@code request.timeout.ms} expires.
+ * the amount of time allocated by {@code default.api.timeout.ms} expires.
*/
@Override
public List partitionsFor(String topic) {
- return partitionsFor(topic, Duration.ofMillis(requestTimeoutMs));
+ return partitionsFor(topic, Duration.ofMillis(defaultApiTimeoutMs));
}
/**
@@ -1774,11 +1791,11 @@ public List partitionsFor(String topic, Duration timeout) {
* this function is called
* @throws org.apache.kafka.common.KafkaException for any other unrecoverable errors
* @throws org.apache.kafka.common.errors.TimeoutException if the offset metadata could not be fetched before
- * the amount of time allocated by {@code request.timeout.ms} expires.
+ * the amount of time allocated by {@code default.api.timeout.ms} expires.
*/
@Override
public Map> listTopics() {
- return listTopics(Duration.ofMillis(requestTimeoutMs));
+ return listTopics(Duration.ofMillis(defaultApiTimeoutMs));
}
/**
@@ -1879,13 +1896,13 @@ public Set paused() {
* @throws org.apache.kafka.common.errors.AuthorizationException if not authorized to the topic(s). See the exception for more details
* @throws IllegalArgumentException if the target timestamp is negative
* @throws org.apache.kafka.common.errors.TimeoutException if the offset metadata could not be fetched before
- * the amount of time allocated by {@code request.timeout.ms} expires.
+ * the amount of time allocated by {@code default.api.timeout.ms} expires.
* @throws org.apache.kafka.common.errors.UnsupportedVersionException if the broker does not support looking up
* the offsets by timestamp
*/
@Override
public Map offsetsForTimes(Map timestampsToSearch) {
- return offsetsForTimes(timestampsToSearch, Duration.ofMillis(requestTimeoutMs));
+ return offsetsForTimes(timestampsToSearch, Duration.ofMillis(defaultApiTimeoutMs));
}
/**
@@ -1939,11 +1956,11 @@ public Map offsetsForTimes(Map beginningOffsets(Collection partitions) {
- return beginningOffsets(partitions, Duration.ofMillis(requestTimeoutMs));
+ return beginningOffsets(partitions, Duration.ofMillis(defaultApiTimeoutMs));
}
/**
@@ -2062,7 +2079,7 @@ public void close() {
@Deprecated
@Override
public void close(long timeout, TimeUnit timeUnit) {
- close(Duration.ofMillis(TimeUnit.MILLISECONDS.toMillis(timeout)));
+ close(Duration.ofMillis(timeUnit.toMillis(timeout)));
}
/**
@@ -2125,12 +2142,12 @@ private void close(long timeoutMs, boolean swallowException) {
firstException.compareAndSet(null, t);
log.error("Failed to close coordinator", t);
}
- ClientUtils.closeQuietly(fetcher, "fetcher", firstException);
- ClientUtils.closeQuietly(interceptors, "consumer interceptors", firstException);
- ClientUtils.closeQuietly(metrics, "consumer metrics", firstException);
- ClientUtils.closeQuietly(client, "consumer network client", firstException);
- ClientUtils.closeQuietly(keyDeserializer, "consumer key deserializer", firstException);
- ClientUtils.closeQuietly(valueDeserializer, "consumer value deserializer", firstException);
+ Utils.closeQuietly(fetcher, "fetcher", firstException);
+ Utils.closeQuietly(interceptors, "consumer interceptors", firstException);
+ Utils.closeQuietly(metrics, "consumer metrics", firstException);
+ Utils.closeQuietly(client, "consumer network client", firstException);
+ Utils.closeQuietly(keyDeserializer, "consumer key deserializer", firstException);
+ Utils.closeQuietly(valueDeserializer, "consumer value deserializer", firstException);
AppInfoParser.unregisterAppInfo(JMX_PREFIX, clientId, metrics);
log.debug("Kafka consumer has been closed");
Throwable exception = firstException.get();
diff --git a/clients/src/main/java/org/apache/kafka/clients/consumer/MockConsumer.java b/clients/src/main/java/org/apache/kafka/clients/consumer/MockConsumer.java
index cf1b07fabe21b..7f42d60bfe813 100644
--- a/clients/src/main/java/org/apache/kafka/clients/consumer/MockConsumer.java
+++ b/clients/src/main/java/org/apache/kafka/clients/consumer/MockConsumer.java
@@ -58,7 +58,8 @@ public class MockConsumer implements Consumer {
private final Set paused;
private Map>> records;
- private KafkaException exception;
+ private KafkaException pollException;
+ private KafkaException offsetsException;
private AtomicBoolean wakeup;
private boolean closed;
@@ -71,7 +72,7 @@ public MockConsumer(OffsetResetStrategy offsetResetStrategy) {
this.beginningOffsets = new HashMap<>();
this.endOffsets = new HashMap<>();
this.pollTasks = new LinkedList<>();
- this.exception = null;
+ this.pollException = null;
this.wakeup = new AtomicBoolean(false);
this.committed = new HashMap<>();
}
@@ -170,9 +171,9 @@ public synchronized ConsumerRecords poll(final Duration timeout) {
throw new WakeupException();
}
- if (exception != null) {
- RuntimeException exception = this.exception;
- this.exception = null;
+ if (pollException != null) {
+ RuntimeException exception = this.pollException;
+ this.pollException = null;
throw exception;
}
@@ -183,16 +184,17 @@ public synchronized ConsumerRecords poll(final Duration timeout) {
// update the consumed offset
final Map>> results = new HashMap<>();
- for (final TopicPartition topicPartition : records.keySet()) {
- results.put(topicPartition, new ArrayList>());
- }
for (Map.Entry>> entry : this.records.entrySet()) {
if (!subscriptions.isPaused(entry.getKey())) {
final List> recs = entry.getValue();
for (final ConsumerRecord rec : recs) {
+ if (beginningOffsets.get(entry.getKey()) != null && beginningOffsets.get(entry.getKey()) > subscriptions.position(entry.getKey())) {
+ throw new OffsetOutOfRangeException(Collections.singletonMap(entry.getKey(), subscriptions.position(entry.getKey())));
+ }
+
if (assignment().contains(entry.getKey()) && rec.offset() >= subscriptions.position(entry.getKey())) {
- results.get(entry.getKey()).add(rec);
+ results.computeIfAbsent(entry.getKey(), partition -> new ArrayList<>()).add(rec);
subscriptions.position(entry.getKey(), rec.offset() + 1);
}
}
@@ -216,8 +218,20 @@ public synchronized void addRecord(ConsumerRecord record) {
recs.add(record);
}
+ /**
+ * @deprecated Use {@link #setPollException(KafkaException)} instead
+ */
+ @Deprecated
public synchronized void setException(KafkaException exception) {
- this.exception = exception;
+ setPollException(exception);
+ }
+
+ public synchronized void setPollException(KafkaException exception) {
+ this.pollException = exception;
+ }
+
+ public synchronized void setOffsetsException(KafkaException exception) {
+ this.offsetsException = exception;
}
@Override
@@ -385,6 +399,11 @@ public synchronized Map offsetsForTimes(Map<
@Override
public synchronized Map beginningOffsets(Collection partitions) {
+ if (offsetsException != null) {
+ RuntimeException exception = this.offsetsException;
+ this.offsetsException = null;
+ throw exception;
+ }
Map result = new HashMap<>();
for (TopicPartition tp : partitions) {
Long beginningOffset = beginningOffsets.get(tp);
@@ -397,6 +416,11 @@ public synchronized Map beginningOffsets(Collection endOffsets(Collection partitions) {
+ if (offsetsException != null) {
+ RuntimeException exception = this.offsetsException;
+ this.offsetsException = null;
+ throw exception;
+ }
Map result = new HashMap<>();
for (TopicPartition tp : partitions) {
Long endOffset = getEndOffset(endOffsets.get(tp));
diff --git a/clients/src/main/java/org/apache/kafka/clients/consumer/RoundRobinAssignor.java b/clients/src/main/java/org/apache/kafka/clients/consumer/RoundRobinAssignor.java
index 7e8d6f2c8077e..3b543f7130832 100644
--- a/clients/src/main/java/org/apache/kafka/clients/consumer/RoundRobinAssignor.java
+++ b/clients/src/main/java/org/apache/kafka/clients/consumer/RoundRobinAssignor.java
@@ -48,7 +48,7 @@
* with 1, 2, and 3 partitions, respectively. Therefore, the partitions are t0p0, t1p0, t1p1, t2p0,
* t2p1, t2p2. C0 is subscribed to t0; C1 is subscribed to t0, t1; and C2 is subscribed to t0, t1, t2.
*
- * Tha assignment will be:
+ * That assignment will be:
* C0: [t0p0]
* C1: [t1p0]
* C2: [t1p1, t2p0, t2p1, t2p2]
diff --git a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/AbstractCoordinator.java b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/AbstractCoordinator.java
index adbaae776ab39..53834fb81dfba 100644
--- a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/AbstractCoordinator.java
+++ b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/AbstractCoordinator.java
@@ -129,7 +129,7 @@ public AbstractCoordinator(LogContext logContext,
String groupId,
int rebalanceTimeoutMs,
int sessionTimeoutMs,
- int heartbeatIntervalMs,
+ Heartbeat heartbeat,
Metrics metrics,
String metricGrpPrefix,
Time time,
@@ -142,11 +142,27 @@ public AbstractCoordinator(LogContext logContext,
this.rebalanceTimeoutMs = rebalanceTimeoutMs;
this.sessionTimeoutMs = sessionTimeoutMs;
this.leaveGroupOnClose = leaveGroupOnClose;
- this.heartbeat = new Heartbeat(sessionTimeoutMs, heartbeatIntervalMs, rebalanceTimeoutMs, retryBackoffMs);
+ this.heartbeat = heartbeat;
this.sensors = new GroupCoordinatorMetrics(metrics, metricGrpPrefix);
this.retryBackoffMs = retryBackoffMs;
}
+ public AbstractCoordinator(LogContext logContext,
+ ConsumerNetworkClient client,
+ String groupId,
+ int rebalanceTimeoutMs,
+ int sessionTimeoutMs,
+ int heartbeatIntervalMs,
+ Metrics metrics,
+ String metricGrpPrefix,
+ Time time,
+ long retryBackoffMs,
+ boolean leaveGroupOnClose) {
+ this(logContext, client, groupId, rebalanceTimeoutMs, sessionTimeoutMs,
+ new Heartbeat(sessionTimeoutMs, heartbeatIntervalMs, rebalanceTimeoutMs, retryBackoffMs),
+ metrics, metricGrpPrefix, time, retryBackoffMs, leaveGroupOnClose);
+ }
+
/**
* Unique identifier for the class of supported protocols (e.g. "consumer" or "connect").
* @return Non-null protocol type name
@@ -184,9 +200,8 @@ protected abstract Map performAssignment(String leaderId,
Map allMemberMetadata);
/**
- * Invoked when a group member has successfully joined a group. If this call is woken up (i.e.
- * if the invocation raises {@link org.apache.kafka.common.errors.WakeupException}), then it
- * will be retried on the next call to {@link #ensureActiveGroup()}.
+ * Invoked when a group member has successfully joined a group. If this call fails with an exception,
+ * then it will be retried using the same assignment state on the next call to {@link #ensureActiveGroup()}.
*
* @param generation The generation that was joined
* @param memberId The identifier for the local member in the group
@@ -319,7 +334,11 @@ public void ensureActiveGroup() {
* @return true iff the group is active
*/
boolean ensureActiveGroup(final long timeoutMs) {
- final long startTime = time.milliseconds();
+ return ensureActiveGroup(timeoutMs, time.milliseconds());
+ }
+
+ // Visible for testing
+ boolean ensureActiveGroup(long timeoutMs, long startMs) {
// always ensure that the coordinator is ready because we may have been disconnected
// when sending heartbeats and does not necessarily require us to rejoin the group.
if (!ensureCoordinatorReady(timeoutMs)) {
@@ -328,7 +347,9 @@ boolean ensureActiveGroup(final long timeoutMs) {
startHeartbeatThreadIfNeeded();
- return joinGroupIfNeeded(remainingTimeAtLeastZero(timeoutMs, time.milliseconds() - startTime));
+ long joinStartMs = time.milliseconds();
+ long joinTimeoutMs = remainingTimeAtLeastZero(timeoutMs, joinStartMs - startMs);
+ return joinGroupIfNeeded(joinTimeoutMs, joinStartMs);
}
private synchronized void startHeartbeatThreadIfNeeded() {
@@ -366,17 +387,17 @@ private void closeHeartbeatThread() {
* Visible for testing.
*
* @param timeoutMs Time to complete this action
+ * @param startTimeMs Current time when invoked
* @return true iff the operation succeeded
*/
- boolean joinGroupIfNeeded(final long timeoutMs) {
- final long startTime = time.milliseconds();
+ boolean joinGroupIfNeeded(final long timeoutMs, final long startTimeMs) {
long elapsedTime = 0L;
while (rejoinNeededOrPending()) {
if (!ensureCoordinatorReady(remainingTimeAtLeastZero(timeoutMs, elapsedTime))) {
return false;
}
- elapsedTime = time.milliseconds() - startTime;
+ elapsedTime = time.milliseconds() - startTimeMs;
// call onJoinPrepare if needed. We set a flag to make sure that we do not call it a second
// time if the client is woken up before a pending rebalance completes. This must be called
@@ -396,7 +417,9 @@ boolean joinGroupIfNeeded(final long timeoutMs) {
}
if (future.succeeded()) {
- onJoinComplete(generation.generationId, generation.memberId, generation.protocol, future.value());
+ // Duplicate the buffer in case `onJoinComplete` does not complete and needs to be retried.
+ ByteBuffer memberAssignment = future.value().duplicate();
+ onJoinComplete(generation.generationId, generation.memberId, generation.protocol, memberAssignment);
// We reset the join group future only after the completion callback returns. This ensures
// that if the callback is woken up, we will retry it on the next joinGroupIfNeeded.
@@ -415,7 +438,7 @@ else if (!future.isRetriable())
}
if (rejoinNeededOrPending()) {
- elapsedTime = time.milliseconds() - startTime;
+ elapsedTime = time.milliseconds() - startTimeMs;
}
}
return true;
@@ -473,9 +496,12 @@ public void onFailure(RuntimeException e) {
* Join the group and return the assignment for the next generation. This function handles both
* JoinGroup and SyncGroup, delegating to {@link #performAssignment(String, String, Map)} if
* elected leader by the coordinator.
+ *
+ * NOTE: This is visible only for testing
+ *
* @return A request future which wraps the assignment returned from the group leader
*/
- private RequestFuture sendJoinGroupRequest() {
+ RequestFuture sendJoinGroupRequest() {
if (coordinatorUnknown())
return RequestFuture.coordinatorNotAvailable();
@@ -489,7 +515,12 @@ private RequestFuture sendJoinGroupRequest() {
metadata()).setRebalanceTimeout(this.rebalanceTimeoutMs);
log.debug("Sending JoinGroup ({}) to coordinator {}", requestBuilder, this.coordinator);
- return client.send(coordinator, requestBuilder)
+
+ // Note that we override the request timeout using the rebalance timeout since that is the
+ // maximum time that it may block on the coordinator. We add an extra 5 seconds for small delays.
+
+ int joinGroupTimeoutMs = Math.max(rebalanceTimeoutMs, rebalanceTimeoutMs + 5000);
+ return client.send(coordinator, requestBuilder, joinGroupTimeoutMs)
.compose(new JoinGroupResponseHandler());
}
diff --git a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/ConsumerCoordinator.java b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/ConsumerCoordinator.java
index 9c19af1703779..f078f8956aef0 100644
--- a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/ConsumerCoordinator.java
+++ b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/ConsumerCoordinator.java
@@ -118,7 +118,7 @@ public ConsumerCoordinator(LogContext logContext,
String groupId,
int rebalanceTimeoutMs,
int sessionTimeoutMs,
- int heartbeatIntervalMs,
+ Heartbeat heartbeat,
List assignors,
Metadata metadata,
SubscriptionState subscriptions,
@@ -136,7 +136,7 @@ public ConsumerCoordinator(LogContext logContext,
groupId,
rebalanceTimeoutMs,
sessionTimeoutMs,
- heartbeatIntervalMs,
+ heartbeat,
metrics,
metricGrpPrefix,
time,
@@ -264,10 +264,10 @@ protected void onJoinComplete(int generation,
this.joinedSubscription = newJoinedSubscription;
}
- // update the metadata and enforce a refresh to make sure the fetcher can start
- // fetching data in the next iteration
+ // Update the metadata to include the full group subscription. The leader will trigger a rebalance
+ // if there are any metadata changes affecting any of the consumed partitions (whether or not this
+ // instance is subscribed to the topics).
this.metadata.setTopics(subscriptions.groupSubscription());
- if (!client.ensureFreshMetadata(Long.MAX_VALUE)) throw new TimeoutException();
// give the assignor a chance to update internal state based on the received assignment
assignor.onAssignment(assignment);
@@ -306,13 +306,16 @@ public boolean poll(final long timeoutMs) {
invokeCompletedOffsetCommitCallbacks();
if (subscriptions.partitionsAutoAssigned()) {
+ // Always update the heartbeat last poll time so that the heartbeat thread does not leave the
+ // group proactively due to application inactivity even if (say) the coordinator cannot be found.
+ pollHeartbeat(currentTime);
+
if (coordinatorUnknown()) {
if (!ensureCoordinatorReady(remainingTimeAtLeastZero(timeoutMs, elapsed))) {
return false;
}
currentTime = time.milliseconds();
elapsed = currentTime - startTime;
-
}
if (rejoinNeededOrPending()) {
@@ -320,6 +323,16 @@ public boolean poll(final long timeoutMs) {
// we need to ensure that the metadata is fresh before joining initially. This ensures
// that we have matched the pattern against the cluster's topics at least once before joining.
if (subscriptions.hasPatternSubscription()) {
+ // For consumer group that uses pattern-based subscription, after a topic is created,
+ // any consumer that discovers the topic after metadata refresh can trigger rebalance
+ // across the entire consumer group. Multiple rebalances can be triggered after one topic
+ // creation if consumers refresh metadata at vastly different times. We can significantly
+ // reduce the number of rebalances caused by single topic creation by asking consumer to
+ // refresh metadata before re-joining the group as long as the refresh backoff time has
+ // passed.
+ if (this.metadata.timeToAllowUpdate(currentTime) == 0) {
+ this.metadata.requestUpdate();
+ }
if (!client.ensureFreshMetadata(remainingTimeAtLeastZero(timeoutMs, elapsed))) {
return false;
}
@@ -333,8 +346,6 @@ public boolean poll(final long timeoutMs) {
currentTime = time.milliseconds();
}
-
- pollHeartbeat(currentTime);
} else {
// For manually assigned partitions, if there are no ready nodes, await metadata.
// If connections to all nodes fail, wakeups triggered while attempting to send fetch
@@ -829,7 +840,11 @@ public void handle(OffsetCommitResponse commitResponse, RequestFuture futu
if (error == Errors.NONE) {
log.debug("Committed offset {} for partition {}", offset, tp);
} else {
- log.error("Offset commit failed on partition {} at offset {}: {}", tp, offset, error.message());
+ if (error.exception() instanceof RetriableException) {
+ log.warn("Offset commit failed on partition {} at offset {}: {}", tp, offset, error.message());
+ } else {
+ log.error("Offset commit failed on partition {} at offset {}: {}", tp, offset, error.message());
+ }
if (error == Errors.GROUP_AUTHORIZATION_FAILED) {
future.raise(new GroupAuthorizationException(groupId));
@@ -964,7 +979,8 @@ private ConsumerCoordinatorMetrics(Metrics metrics, String metricGrpPrefix) {
Measurable numParts =
new Measurable() {
public double measure(MetricConfig config, long now) {
- return subscriptions.assignedPartitions().size();
+ // Get the number of assigned partitions in a thread safe manner
+ return subscriptions.numAssignedPartitions();
}
};
metrics.addMetric(metrics.metricName("assigned-partitions",
diff --git a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/ConsumerNetworkClient.java b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/ConsumerNetworkClient.java
index a9e167a9a2e7f..0bf0aad1b945c 100644
--- a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/ConsumerNetworkClient.java
+++ b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/ConsumerNetworkClient.java
@@ -62,7 +62,7 @@ public class ConsumerNetworkClient implements Closeable {
private final Time time;
private final long retryBackoffMs;
private final int maxPollTimeoutMs;
- private final long unsentExpiryMs;
+ private final int requestTimeoutMs;
private final AtomicBoolean wakeupDisabled = new AtomicBoolean();
// We do not need high throughput, so use a fair lock to try to avoid starvation
@@ -83,7 +83,7 @@ public ConsumerNetworkClient(LogContext logContext,
Metadata metadata,
Time time,
long retryBackoffMs,
- long requestTimeoutMs,
+ int requestTimeoutMs,
int maxPollTimeoutMs) {
this.log = logContext.logger(ConsumerNetworkClient.class);
this.client = client;
@@ -91,7 +91,15 @@ public ConsumerNetworkClient(LogContext logContext,
this.time = time;
this.retryBackoffMs = retryBackoffMs;
this.maxPollTimeoutMs = Math.min(maxPollTimeoutMs, MAX_POLL_TIMEOUT_MS);
- this.unsentExpiryMs = requestTimeoutMs;
+ this.requestTimeoutMs = requestTimeoutMs;
+ }
+
+
+ /**
+ * Send a request with the default timeout. See {@link #send(Node, AbstractRequest.Builder, int)}.
+ */
+ public RequestFuture send(Node node, AbstractRequest.Builder> requestBuilder) {
+ return send(node, requestBuilder, requestTimeoutMs);
}
/**
@@ -104,13 +112,18 @@ public ConsumerNetworkClient(LogContext logContext,
*
* @param node The destination of the request
* @param requestBuilder A builder for the request payload
+ * @param requestTimeoutMs Maximum time in milliseconds to await a response before disconnecting the socket and
+ * cancelling the request. The request may be cancelled sooner if the socket disconnects
+ * for any reason.
* @return A future which indicates the result of the send.
*/
- public RequestFuture send(Node node, AbstractRequest.Builder> requestBuilder) {
+ public RequestFuture send(Node node,
+ AbstractRequest.Builder> requestBuilder,
+ int requestTimeoutMs) {
long now = time.milliseconds();
RequestFutureCompletionHandler completionHandler = new RequestFutureCompletionHandler();
ClientRequest clientRequest = client.newClientRequest(node.idString(), requestBuilder, now, true,
- completionHandler);
+ requestTimeoutMs, completionHandler);
unsent.put(node, clientRequest);
// wakeup the client in case it is blocking in poll so that we can send the queued request
@@ -136,13 +149,6 @@ public boolean hasReadyNodes(long now) {
}
}
- /**
- * Block until the metadata has been refreshed.
- */
- public void awaitMetadataUpdate() {
- awaitMetadataUpdate(Long.MAX_VALUE);
- }
-
/**
* Block waiting on the metadata refresh with a timeout.
*
@@ -444,10 +450,10 @@ public void disconnectAsync(Node node) {
private void failExpiredRequests(long now) {
// clear all expired unsent requests and fail their corresponding futures
- Collection expiredRequests = unsent.removeExpiredRequests(now, unsentExpiryMs);
+ Collection expiredRequests = unsent.removeExpiredRequests(now);
for (ClientRequest request : expiredRequests) {
RequestFutureCompletionHandler handler = (RequestFutureCompletionHandler) request.callback();
- handler.onFailure(new TimeoutException("Failed to send request after " + unsentExpiryMs + " ms."));
+ handler.onFailure(new TimeoutException("Failed to send request after " + request.requestTimeoutMs() + " ms."));
}
}
@@ -655,13 +661,14 @@ public boolean hasRequests() {
return false;
}
- public Collection removeExpiredRequests(long now, long unsentExpiryMs) {
+ private Collection removeExpiredRequests(long now) {
List expiredRequests = new ArrayList<>();
for (ConcurrentLinkedQueue requests : unsent.values()) {
Iterator requestIterator = requests.iterator();
while (requestIterator.hasNext()) {
ClientRequest request = requestIterator.next();
- if (request.createdTimeMs() < now - unsentExpiryMs) {
+ long elapsedMs = Math.max(0, now - request.createdTimeMs());
+ if (elapsedMs > request.requestTimeoutMs()) {
expiredRequests.add(request);
requestIterator.remove();
} else
diff --git a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/Fetcher.java b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/Fetcher.java
index ca8e0d26c81e2..cf328735a16cb 100644
--- a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/Fetcher.java
+++ b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/Fetcher.java
@@ -91,7 +91,21 @@
import static org.apache.kafka.common.serialization.ExtendedDeserializer.Wrapper.ensureExtended;
/**
- * This class manage the fetching process with the brokers.
+ * This class manages the fetching process with the brokers.
+ *
+ * Thread-safety:
+ * Requests and responses of Fetcher may be processed by different threads since heartbeat
+ * thread may process responses. Other operations are single-threaded and invoked only from
+ * the thread polling the consumer.
+ *
+ * - If a response handler accesses any shared state of the Fetcher (e.g. FetchSessionHandler),
+ * all access to that state must be synchronized on the Fetcher instance.
+ * - If a response handler accesses any shared state of the coordinator (e.g. SubscriptionState),
+ * it is assumed that all access to that state is synchronized on the coordinator instance by
+ * the caller.
+ * - Responses that collate partial responses from multiple brokers (e.g. to list offsets) are
+ * synchronized on the response future.
+ *
*/
public class Fetcher implements SubscriptionState.Listener, Closeable {
private final Logger log;
@@ -187,7 +201,7 @@ public boolean hasCompletedFetches() {
* an in-flight fetch or pending fetch data.
* @return number of fetches sent
*/
- public int sendFetches() {
+ public synchronized int sendFetches() {
Map fetchRequestMap = prepareFetchRequests();
for (Map.Entry entry : fetchRequestMap.entrySet()) {
final Node fetchTarget = entry.getKey();
@@ -205,39 +219,43 @@ public int sendFetches() {
.addListener(new RequestFutureListener() {
@Override
public void onSuccess(ClientResponse resp) {
- FetchResponse response = (FetchResponse) resp.responseBody();
- FetchSessionHandler handler = sessionHandlers.get(fetchTarget.id());
- if (handler == null) {
- log.error("Unable to find FetchSessionHandler for node {}. Ignoring fetch response.",
- fetchTarget.id());
- return;
+ synchronized (Fetcher.this) {
+ FetchResponse response = (FetchResponse) resp.responseBody();
+ FetchSessionHandler handler = sessionHandler(fetchTarget.id());
+ if (handler == null) {
+ log.error("Unable to find FetchSessionHandler for node {}. Ignoring fetch response.",
+ fetchTarget.id());
+ return;
+ }
+ if (!handler.handleResponse(response)) {
+ return;
+ }
+
+ Set partitions = new HashSet<>(response.responseData().keySet());
+ FetchResponseMetricAggregator metricAggregator = new FetchResponseMetricAggregator(sensors, partitions);
+
+ for (Map.Entry> entry : response.responseData().entrySet()) {
+ TopicPartition partition = entry.getKey();
+ long fetchOffset = data.sessionPartitions().get(partition).fetchOffset;
+ FetchResponse.PartitionData fetchData = entry.getValue();
+
+ log.debug("Fetch {} at offset {} for partition {} returned fetch data {}",
+ isolationLevel, fetchOffset, partition, fetchData);
+ completedFetches.add(new CompletedFetch(partition, fetchOffset, fetchData, metricAggregator,
+ resp.requestHeader().apiVersion()));
+ }
+
+ sensors.fetchLatency.record(resp.requestLatencyMs());
}
- if (!handler.handleResponse(response)) {
- return;
- }
-
- Set partitions = new HashSet<>(response.responseData().keySet());
- FetchResponseMetricAggregator metricAggregator = new FetchResponseMetricAggregator(sensors, partitions);
-
- for (Map.Entry> entry : response.responseData().entrySet()) {
- TopicPartition partition = entry.getKey();
- long fetchOffset = data.sessionPartitions().get(partition).fetchOffset;
- FetchResponse.PartitionData fetchData = entry.getValue();
-
- log.debug("Fetch {} at offset {} for partition {} returned fetch data {}",
- isolationLevel, fetchOffset, partition, fetchData);
- completedFetches.add(new CompletedFetch(partition, fetchOffset, fetchData, metricAggregator,
- resp.requestHeader().apiVersion()));
- }
-
- sensors.fetchLatency.record(resp.requestLatencyMs());
}
@Override
public void onFailure(RuntimeException e) {
- FetchSessionHandler handler = sessionHandlers.get(fetchTarget.id());
- if (handler != null) {
- handler.handleError(e);
+ synchronized (Fetcher.this) {
+ FetchSessionHandler handler = sessionHandler(fetchTarget.id());
+ if (handler != null) {
+ handler.handleError(e);
+ }
}
}
});
@@ -313,7 +331,7 @@ else if (error.exception() instanceof RetriableException)
if (!shouldRetry) {
HashMap> topicsPartitionInfos = new HashMap<>();
for (String topic : cluster.topics())
- topicsPartitionInfos.put(topic, cluster.availablePartitionsForTopic(topic));
+ topicsPartitionInfos.put(topic, cluster.partitionsForTopic(topic));
return topicsPartitionInfos;
}
}
@@ -422,7 +440,7 @@ private ListOffsetResult fetchOffsetsByTimes(Map timestamp
if (value.partitionsToRetry.isEmpty())
return result;
- remainingToSearch.keySet().removeAll(result.fetchedOffsets.keySet());
+ remainingToSearch.keySet().retainAll(value.partitionsToRetry);
} else if (!future.isRetriable()) {
throw future.exception();
}
@@ -590,7 +608,8 @@ private void resetOffsetsAsync(Map partitionResetTimestamp
for (TopicPartition tp : partitionResetTimestamps.keySet())
metadata.add(tp.topic());
- Map> timestampsToSearchByNode = groupListOffsetRequests(partitionResetTimestamps);
+ Map> timestampsToSearchByNode =
+ groupListOffsetRequests(partitionResetTimestamps, new HashSet<>());
for (Map.Entry> entry : timestampsToSearchByNode.entrySet()) {
Node node = entry.getKey();
final Map resetTimestamps = entry.getValue();
@@ -639,18 +658,19 @@ private RequestFuture sendListOffsetsRequests(final Map> timestampsToSearchByNode = groupListOffsetRequests(timestampsToSearch);
+ final Set partitionsToRetry = new HashSet<>();
+ Map> timestampsToSearchByNode =
+ groupListOffsetRequests(timestampsToSearch, partitionsToRetry);
if (timestampsToSearchByNode.isEmpty())
return RequestFuture.failure(new StaleMetadataException());
final RequestFuture listOffsetRequestsFuture = new RequestFuture<>();
final Map fetchedTimestampOffsets = new HashMap<>();
- final Set partitionsToRetry = new HashSet<>();
final AtomicInteger remainingResponses = new AtomicInteger(timestampsToSearchByNode.size());
for (Map.Entry> entry : timestampsToSearchByNode.entrySet()) {
RequestFuture future =
- sendListOffsetRequest(entry.getKey(), entry.getValue(), requireTimestamps);
+ sendListOffsetRequest(entry.getKey(), entry.getValue(), requireTimestamps);
future.addListener(new RequestFutureListener() {
@Override
public void onSuccess(ListOffsetResult partialResult) {
@@ -677,7 +697,16 @@ public void onFailure(RuntimeException e) {
return listOffsetRequestsFuture;
}
- private Map> groupListOffsetRequests(Map timestampsToSearch) {
+ /**
+ * Groups timestamps to search by node for topic partitions in `timestampsToSearch` that have
+ * leaders available. Topic partitions from `timestampsToSearch` that do not have their leader
+ * available are added to `partitionsToRetry`
+ * @param timestampsToSearch The mapping from partitions ot the target timestamps
+ * @param partitionsToRetry A set of topic partitions that will be extended with partitions
+ * that need metadata update or re-connect to the leader.
+ */
+ private Map> groupListOffsetRequests(
+ Map timestampsToSearch, Set partitionsToRetry) {
final Map> timestampsToSearchByNode = new HashMap<>();
for (Map.Entry entry: timestampsToSearch.entrySet()) {
TopicPartition tp = entry.getKey();
@@ -686,9 +715,11 @@ private Map> groupListOffsetRequests(Map> groupListOffsetRequests(Map topicData = timestampsToSearchByNode.get(node);
@@ -866,7 +898,7 @@ private Map prepareFetchRequests() {
// if there is a leader and no in-flight requests, issue a new fetch
FetchSessionHandler.Builder builder = fetchable.get(node);
if (builder == null) {
- FetchSessionHandler handler = sessionHandlers.get(node.id());
+ FetchSessionHandler handler = sessionHandler(node.id());
if (handler == null) {
handler = new FetchSessionHandler(logContext, node.id());
sessionHandlers.put(node.id(), handler);
@@ -1023,6 +1055,11 @@ public void onAssignment(Set assignment) {
sensors.updatePartitionLagAndLeadSensors(assignment);
}
+ // Visibilty for testing
+ protected FetchSessionHandler sessionHandler(int node) {
+ return sessionHandlers.get(node);
+ }
+
public static Sensor throttleTimeSensor(Metrics metrics, FetcherMetricsRegistry metricsRegistry) {
Sensor fetchThrottleTimeSensor = metrics.sensor("fetch-throttle-time");
fetchThrottleTimeSensor.add(metrics.metricInstance(metricsRegistry.fetchThrottleTimeAvg), new Avg());
@@ -1242,8 +1279,7 @@ private boolean containsAbortMarker(RecordBatch batch) {
Iterator batchIterator = batch.iterator();
if (!batchIterator.hasNext())
- throw new InvalidRecordException("Invalid batch for partition " + partition + " at offset " +
- batch.baseOffset() + " with control sequence set, but no records");
+ return false;
Record firstRecord = batchIterator.next();
return ControlRecordType.ABORT == ControlRecordType.parse(firstRecord.key());
@@ -1447,16 +1483,6 @@ private void recordPartitionLag(TopicPartition tp, long lag) {
recordsLag.add(this.metrics.metricInstance(metricsRegistry.partitionRecordsLag, metricTags), new Value());
recordsLag.add(this.metrics.metricInstance(metricsRegistry.partitionRecordsLagMax, metricTags), new Max());
recordsLag.add(this.metrics.metricInstance(metricsRegistry.partitionRecordsLagAvg, metricTags), new Avg());
-
- recordsLag.add(this.metrics.metricName(name,
- metricsRegistry.partitionRecordsLagDeprecated.group(),
- metricsRegistry.partitionRecordsLagDeprecated.description()), new Value());
- recordsLag.add(this.metrics.metricName(name + "-max",
- metricsRegistry.partitionRecordsLagMaxDeprecated.group(),
- metricsRegistry.partitionRecordsLagMaxDeprecated.description()), new Max());
- recordsLag.add(this.metrics.metricName(name + "-avg",
- metricsRegistry.partitionRecordsLagAvgDeprecated.group(),
- metricsRegistry.partitionRecordsLagAvgDeprecated.description()), new Avg());
}
recordsLag.record(lag);
}
diff --git a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/FetcherMetricsRegistry.java b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/FetcherMetricsRegistry.java
index 6eb4fa20ff48f..f86961545cc99 100644
--- a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/FetcherMetricsRegistry.java
+++ b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/FetcherMetricsRegistry.java
@@ -54,10 +54,6 @@ public class FetcherMetricsRegistry {
public MetricNameTemplate partitionRecordsLead;
public MetricNameTemplate partitionRecordsLeadMin;
public MetricNameTemplate partitionRecordsLeadAvg;
- // To remove in 2.0
- public MetricNameTemplate partitionRecordsLagDeprecated;
- public MetricNameTemplate partitionRecordsLagMaxDeprecated;
- public MetricNameTemplate partitionRecordsLagAvgDeprecated;
public FetcherMetricsRegistry() {
this(new HashSet(), "");
@@ -68,72 +64,65 @@ public FetcherMetricsRegistry(String metricGrpPrefix) {
}
public FetcherMetricsRegistry(Set tags, String metricGrpPrefix) {
-
+
/***** Client level *****/
String groupName = metricGrpPrefix + "-fetch-manager-metrics";
-
- this.fetchSizeAvg = new MetricNameTemplate("fetch-size-avg", groupName,
+
+ this.fetchSizeAvg = new MetricNameTemplate("fetch-size-avg", groupName,
"The average number of bytes fetched per request", tags);
- this.fetchSizeMax = new MetricNameTemplate("fetch-size-max", groupName,
+ this.fetchSizeMax = new MetricNameTemplate("fetch-size-max", groupName,
"The maximum number of bytes fetched per request", tags);
- this.bytesConsumedRate = new MetricNameTemplate("bytes-consumed-rate", groupName,
+ this.bytesConsumedRate = new MetricNameTemplate("bytes-consumed-rate", groupName,
"The average number of bytes consumed per second", tags);
this.bytesConsumedTotal = new MetricNameTemplate("bytes-consumed-total", groupName,
"The total number of bytes consumed", tags);
- this.recordsPerRequestAvg = new MetricNameTemplate("records-per-request-avg", groupName,
+ this.recordsPerRequestAvg = new MetricNameTemplate("records-per-request-avg", groupName,
"The average number of records in each request", tags);
- this.recordsConsumedRate = new MetricNameTemplate("records-consumed-rate", groupName,
+ this.recordsConsumedRate = new MetricNameTemplate("records-consumed-rate", groupName,
"The average number of records consumed per second", tags);
this.recordsConsumedTotal = new MetricNameTemplate("records-consumed-total", groupName,
"The total number of records consumed", tags);
- this.fetchLatencyAvg = new MetricNameTemplate("fetch-latency-avg", groupName,
+ this.fetchLatencyAvg = new MetricNameTemplate("fetch-latency-avg", groupName,
"The average time taken for a fetch request.", tags);
- this.fetchLatencyMax = new MetricNameTemplate("fetch-latency-max", groupName,
+ this.fetchLatencyMax = new MetricNameTemplate("fetch-latency-max", groupName,
"The max time taken for any fetch request.", tags);
- this.fetchRequestRate = new MetricNameTemplate("fetch-rate", groupName,
+ this.fetchRequestRate = new MetricNameTemplate("fetch-rate", groupName,
"The number of fetch requests per second.", tags);
this.fetchRequestTotal = new MetricNameTemplate("fetch-total", groupName,
"The total number of fetch requests.", tags);
- this.recordsLagMax = new MetricNameTemplate("records-lag-max", groupName,
+ this.recordsLagMax = new MetricNameTemplate("records-lag-max", groupName,
"The maximum lag in terms of number of records for any partition in this window", tags);
this.recordsLeadMin = new MetricNameTemplate("records-lead-min", groupName,
"The minimum lead in terms of number of records for any partition in this window", tags);
- this.fetchThrottleTimeAvg = new MetricNameTemplate("fetch-throttle-time-avg", groupName,
+ this.fetchThrottleTimeAvg = new MetricNameTemplate("fetch-throttle-time-avg", groupName,
"The average throttle time in ms", tags);
- this.fetchThrottleTimeMax = new MetricNameTemplate("fetch-throttle-time-max", groupName,
+ this.fetchThrottleTimeMax = new MetricNameTemplate("fetch-throttle-time-max", groupName,
"The maximum throttle time in ms", tags);
/***** Topic level *****/
Set topicTags = new LinkedHashSet<>(tags);
topicTags.add("topic");
- this.topicFetchSizeAvg = new MetricNameTemplate("fetch-size-avg", groupName,
+ this.topicFetchSizeAvg = new MetricNameTemplate("fetch-size-avg", groupName,
"The average number of bytes fetched per request for a topic", topicTags);
- this.topicFetchSizeMax = new MetricNameTemplate("fetch-size-max", groupName,
+ this.topicFetchSizeMax = new MetricNameTemplate("fetch-size-max", groupName,
"The maximum number of bytes fetched per request for a topic", topicTags);
- this.topicBytesConsumedRate = new MetricNameTemplate("bytes-consumed-rate", groupName,
+ this.topicBytesConsumedRate = new MetricNameTemplate("bytes-consumed-rate", groupName,
"The average number of bytes consumed per second for a topic", topicTags);
this.topicBytesConsumedTotal = new MetricNameTemplate("bytes-consumed-total", groupName,
"The total number of bytes consumed for a topic", topicTags);
- this.topicRecordsPerRequestAvg = new MetricNameTemplate("records-per-request-avg", groupName,
+ this.topicRecordsPerRequestAvg = new MetricNameTemplate("records-per-request-avg", groupName,
"The average number of records in each request for a topic", topicTags);
- this.topicRecordsConsumedRate = new MetricNameTemplate("records-consumed-rate", groupName,
+ this.topicRecordsConsumedRate = new MetricNameTemplate("records-consumed-rate", groupName,
"The average number of records consumed per second for a topic", topicTags);
this.topicRecordsConsumedTotal = new MetricNameTemplate("records-consumed-total", groupName,
"The total number of records consumed for a topic", topicTags);
-
- this.partitionRecordsLagDeprecated = new MetricNameTemplate("{topic}-{partition}.records-lag", groupName,
- "The latest lag of the partition (DEPRECATED use the tag based version instead)", tags);
- this.partitionRecordsLagMaxDeprecated = new MetricNameTemplate("{topic}-{partition}.records-lag-max", groupName,
- "The max lag of the partition (DEPRECATED use the tag based version instead)", tags);
- this.partitionRecordsLagAvgDeprecated = new MetricNameTemplate("{topic}-{partition}.records-lag-avg", groupName,
- "The average lag of the partition (DEPRECATED use the tag based version instead)", tags);
/***** Partition level *****/
Set partitionTags = new HashSet<>(topicTags);
@@ -150,9 +139,9 @@ public FetcherMetricsRegistry(Set tags, String metricGrpPrefix) {
"The min lead of the partition", partitionTags);
this.partitionRecordsLeadAvg = new MetricNameTemplate("records-lead-avg", groupName,
"The average lead of the partition", partitionTags);
-
+
}
-
+
public List getAllTemplates() {
return Arrays.asList(
fetchSizeAvg,
@@ -177,9 +166,6 @@ public List getAllTemplates() {
topicRecordsPerRequestAvg,
topicRecordsConsumedRate,
topicRecordsConsumedTotal,
- partitionRecordsLagDeprecated,
- partitionRecordsLagAvgDeprecated,
- partitionRecordsLagMaxDeprecated,
partitionRecordsLag,
partitionRecordsLagAvg,
partitionRecordsLagMax,
diff --git a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/Heartbeat.java b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/Heartbeat.java
index 38a7c78a599c4..01d78101a1e21 100644
--- a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/Heartbeat.java
+++ b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/Heartbeat.java
@@ -20,9 +20,9 @@
* A helper class for managing the heartbeat to the coordinator
*/
public final class Heartbeat {
- private final long sessionTimeout;
- private final long heartbeatInterval;
- private final long maxPollInterval;
+ private final int sessionTimeoutMs;
+ private final int heartbeatIntervalMs;
+ private final int maxPollIntervalMs;
private final long retryBackoffMs;
private volatile long lastHeartbeatSend; // volatile since it is read by metrics
@@ -31,16 +31,16 @@ public final class Heartbeat {
private long lastPoll;
private boolean heartbeatFailed;
- public Heartbeat(long sessionTimeout,
- long heartbeatInterval,
- long maxPollInterval,
+ public Heartbeat(int sessionTimeoutMs,
+ int heartbeatIntervalMs,
+ int maxPollIntervalMs,
long retryBackoffMs) {
- if (heartbeatInterval >= sessionTimeout)
+ if (heartbeatIntervalMs >= sessionTimeoutMs)
throw new IllegalArgumentException("Heartbeat must be set lower than the session timeout");
- this.sessionTimeout = sessionTimeout;
- this.heartbeatInterval = heartbeatInterval;
- this.maxPollInterval = maxPollInterval;
+ this.sessionTimeoutMs = sessionTimeoutMs;
+ this.heartbeatIntervalMs = heartbeatIntervalMs;
+ this.maxPollIntervalMs = maxPollIntervalMs;
this.retryBackoffMs = retryBackoffMs;
}
@@ -75,7 +75,7 @@ public long timeToNextHeartbeat(long now) {
if (heartbeatFailed)
delayToNextHeartbeat = retryBackoffMs;
else
- delayToNextHeartbeat = heartbeatInterval;
+ delayToNextHeartbeat = heartbeatIntervalMs;
if (timeSinceLastHeartbeat > delayToNextHeartbeat)
return 0;
@@ -84,11 +84,11 @@ public long timeToNextHeartbeat(long now) {
}
public boolean sessionTimeoutExpired(long now) {
- return now - Math.max(lastSessionReset, lastHeartbeatReceive) > sessionTimeout;
+ return now - Math.max(lastSessionReset, lastHeartbeatReceive) > sessionTimeoutMs;
}
public long interval() {
- return heartbeatInterval;
+ return heartbeatIntervalMs;
}
public void resetTimeouts(long now) {
@@ -98,7 +98,11 @@ public void resetTimeouts(long now) {
}
public boolean pollTimeoutExpired(long now) {
- return now - lastPoll > maxPollInterval;
+ return now - lastPoll > maxPollIntervalMs;
+ }
+
+ public long lastPollTime() {
+ return lastPoll;
}
}
\ No newline at end of file
diff --git a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/SubscriptionState.java b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/SubscriptionState.java
index e28973452a7f8..542c413d33e55 100644
--- a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/SubscriptionState.java
+++ b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/SubscriptionState.java
@@ -268,6 +268,14 @@ public Set assignedPartitions() {
return this.assignment.partitionSet();
}
+ /**
+ * Provides the number of assigned partitions in a thread safe manner.
+ * @return the number of assigned partitions.
+ */
+ public int numAssignedPartitions() {
+ return this.assignment.size();
+ }
+
public List fetchablePartitions() {
List fetchable = new ArrayList<>(assignment.size());
for (PartitionStates.PartitionState state : assignment.partitionStates()) {
diff --git a/clients/src/main/java/org/apache/kafka/clients/producer/KafkaProducer.java b/clients/src/main/java/org/apache/kafka/clients/producer/KafkaProducer.java
index a5af5b60093d9..f18f786a03c20 100644
--- a/clients/src/main/java/org/apache/kafka/clients/producer/KafkaProducer.java
+++ b/clients/src/main/java/org/apache/kafka/clients/producer/KafkaProducer.java
@@ -66,6 +66,7 @@
import org.apache.kafka.common.utils.KafkaThread;
import org.apache.kafka.common.utils.LogContext;
import org.apache.kafka.common.utils.Time;
+import org.apache.kafka.common.utils.Utils;
import org.slf4j.Logger;
import java.net.InetSocketAddress;
@@ -789,12 +790,12 @@ public Future send(ProducerRecord record) {
*
* @throws AuthenticationException if authentication fails. See the exception for more details
* @throws AuthorizationException fatal error indicating that the producer is not allowed to write
- * @throws IllegalStateException if a transactional.id has been configured and no transaction has been started
+ * @throws IllegalStateException if a transactional.id has been configured and no transaction has been started, or
+ * when send is invoked after producer has been closed.
* @throws InterruptException If the thread is interrupted while blocked
* @throws SerializationException If the key or value are not valid objects given the configured serializers
* @throws TimeoutException If the time taken for fetching metadata or allocating memory for the record has surpassed max.block.ms.
* @throws KafkaException If a Kafka related error occurs that does not belong to the public API exceptions.
- *
*/
@Override
public Future send(ProducerRecord record, Callback callback) {
@@ -803,14 +804,29 @@ public Future send(ProducerRecord record, Callback callbac
return doSend(interceptedRecord, callback);
}
+ // Verify that this producer instance has not been closed. This method throws IllegalStateException if the producer
+ // has already been closed.
+ private void throwIfProducerClosed() {
+ if (ioThread == null || !ioThread.isAlive())
+ throw new IllegalStateException("Cannot perform operation after producer has been closed");
+ }
+
/**
* Implementation of asynchronously send a record to a topic.
*/
private Future doSend(ProducerRecord record, Callback callback) {
TopicPartition tp = null;
try {
+ throwIfProducerClosed();
// first make sure the metadata for the topic is available
- ClusterAndWaitTime clusterAndWaitTime = waitOnMetadata(record.topic(), record.partition(), maxBlockTimeMs);
+ ClusterAndWaitTime clusterAndWaitTime;
+ try {
+ clusterAndWaitTime = waitOnMetadata(record.topic(), record.partition(), maxBlockTimeMs);
+ } catch (KafkaException e) {
+ if (metadata.isClosed())
+ throw new KafkaException("Producer closed while send in progress", e);
+ throw e;
+ }
long remainingWaitMs = Math.max(0, maxBlockTimeMs - clusterAndWaitTime.waitedOnMetadataMs);
Cluster cluster = clusterAndWaitTime.cluster;
byte[] serializedKey;
@@ -895,6 +911,7 @@ private void setReadOnly(Headers headers) {
* @param partition A specific partition expected to exist in metadata, or null if there's no preference
* @param maxWaitMs The maximum time in ms for waiting on the metadata
* @return The cluster containing topic metadata and the amount of time we waited in ms
+ * @throws KafkaException for all Kafka-related exceptions, including the case where this method is called after producer close
*/
private ClusterAndWaitTime waitOnMetadata(String topic, Integer partition, long maxWaitMs) throws InterruptedException {
// add topic to metadata topic list if it is not there already and reset expiry
@@ -909,12 +926,15 @@ private ClusterAndWaitTime waitOnMetadata(String topic, Integer partition, long
long begin = time.milliseconds();
long remainingWaitMs = maxWaitMs;
long elapsed;
- // Issue metadata requests until we have metadata for the topic or maxWaitTimeMs is exceeded.
- // In case we already have cached metadata for the topic, but the requested partition is greater
- // than expected, issue an update request only once. This is necessary in case the metadata
+ // Issue metadata requests until we have metadata for the topic and the requested partition,
+ // or until maxWaitTimeMs is exceeded. This is necessary in case the metadata
// is stale and the number of partitions for this topic has increased in the meantime.
do {
- log.trace("Requesting metadata update for topic {}.", topic);
+ if (partition != null) {
+ log.trace("Requesting metadata update for partition {} of topic {}.", partition, topic);
+ } else {
+ log.trace("Requesting metadata update for topic {}.", topic);
+ }
metadata.add(topic);
int version = metadata.requestUpdate();
sender.wakeup();
@@ -922,22 +942,24 @@ private ClusterAndWaitTime waitOnMetadata(String topic, Integer partition, long
metadata.awaitUpdate(version, remainingWaitMs);
} catch (TimeoutException ex) {
// Rethrow with original maxWaitMs to prevent logging exception with remainingWaitMs
- throw new TimeoutException("Failed to update metadata after " + maxWaitMs + " ms.");
+ throw new TimeoutException(
+ String.format("Topic %s not present in metadata after %d ms.",
+ topic, maxWaitMs));
}
cluster = metadata.fetch();
elapsed = time.milliseconds() - begin;
- if (elapsed >= maxWaitMs)
- throw new TimeoutException("Failed to update metadata after " + maxWaitMs + " ms.");
+ if (elapsed >= maxWaitMs) {
+ throw new TimeoutException(partitionsCount == null ?
+ String.format("Topic %s not present in metadata after %d ms.",
+ topic, maxWaitMs) :
+ String.format("Partition %d of topic %s with partition count %d is not present in metadata after %d ms.",
+ partition, topic, partitionsCount, maxWaitMs));
+ }
if (cluster.unauthorizedTopics().contains(topic))
throw new TopicAuthorizationException(topic);
remainingWaitMs = maxWaitMs - elapsed;
partitionsCount = cluster.partitionCountForTopic(topic);
- } while (partitionsCount == null);
-
- if (partition != null && partition >= partitionsCount) {
- throw new KafkaException(
- String.format("Invalid partition given with record: %d is not in the range [0...%d).", partition, partitionsCount));
- }
+ } while (partitionsCount == null || (partition != null && partition >= partitionsCount));
return new ClusterAndWaitTime(cluster, elapsed);
}
@@ -1008,8 +1030,9 @@ public void flush() {
* Get the partition metadata for the given topic. This can be used for custom partitioning.
* @throws AuthenticationException if authentication fails. See the exception for more details
* @throws AuthorizationException if not authorized to the specified topic. See the exception for more details
- * @throws InterruptException If the thread is interrupted while blocked
+ * @throws InterruptException if the thread is interrupted while blocked
* @throws TimeoutException if metadata could not be refreshed within {@code max.block.ms}
+ * @throws KafkaException for all Kafka-related exceptions, including the case where this method is called after producer close
*/
@Override
public List partitionsFor(String topic) {
@@ -1107,11 +1130,11 @@ private void close(long timeout, TimeUnit timeUnit, boolean swallowException) {
}
}
- ClientUtils.closeQuietly(interceptors, "producer interceptors", firstException);
- ClientUtils.closeQuietly(metrics, "producer metrics", firstException);
- ClientUtils.closeQuietly(keySerializer, "producer keySerializer", firstException);
- ClientUtils.closeQuietly(valueSerializer, "producer valueSerializer", firstException);
- ClientUtils.closeQuietly(partitioner, "producer partitioner", firstException);
+ Utils.closeQuietly(interceptors, "producer interceptors", firstException);
+ Utils.closeQuietly(metrics, "producer metrics", firstException);
+ Utils.closeQuietly(keySerializer, "producer keySerializer", firstException);
+ Utils.closeQuietly(valueSerializer, "producer valueSerializer", firstException);
+ Utils.closeQuietly(partitioner, "producer partitioner", firstException);
AppInfoParser.unregisterAppInfo(JMX_PREFIX, clientId, metrics);
log.debug("Kafka producer has been closed");
Throwable exception = firstException.get();
diff --git a/clients/src/main/java/org/apache/kafka/clients/producer/MockProducer.java b/clients/src/main/java/org/apache/kafka/clients/producer/MockProducer.java
index 9e9869a7e488c..ab036a0992cb1 100644
--- a/clients/src/main/java/org/apache/kafka/clients/producer/MockProducer.java
+++ b/clients/src/main/java/org/apache/kafka/clients/producer/MockProducer.java
@@ -21,6 +21,7 @@
import org.apache.kafka.clients.producer.internals.FutureRecordMetadata;
import org.apache.kafka.clients.producer.internals.ProduceRequestResult;
import org.apache.kafka.common.Cluster;
+import org.apache.kafka.common.KafkaException;
import org.apache.kafka.common.Metric;
import org.apache.kafka.common.MetricName;
import org.apache.kafka.common.PartitionInfo;
@@ -67,6 +68,7 @@ public class MockProducer implements Producer {
private boolean transactionCommitted;
private boolean transactionAborted;
private boolean producerFenced;
+ private boolean producerFencedOnClose;
private boolean sentOffsets;
private long commitCount = 0L;
private Map mockMetrics;
@@ -206,7 +208,7 @@ public void abortTransaction() throws ProducerFencedException {
this.transactionInFlight = false;
}
- private void verifyProducerState() {
+ private synchronized void verifyProducerState() {
if (this.closed) {
throw new IllegalStateException("MockProducer is already closed.");
}
@@ -244,7 +246,12 @@ public synchronized Future send(ProducerRecord record) {
*/
@Override
public synchronized Future send(ProducerRecord record, Callback callback) {
- verifyProducerState();
+ if (this.closed) {
+ throw new IllegalStateException("MockProducer is already closed.");
+ }
+ if (this.producerFenced) {
+ throw new KafkaException("MockProducer is fenced.", new ProducerFencedException("Fenced"));
+ }
int partition = 0;
if (!this.cluster.partitionsForTopic(record.topic()).isEmpty())
partition = partition(record, this.cluster);
@@ -311,8 +318,8 @@ public void close() {
@Override
public void close(long timeout, TimeUnit timeUnit) {
- if (this.closed) {
- throw new IllegalStateException("MockProducer is already closed.");
+ if (producerFencedOnClose) {
+ throw new ProducerFencedException("MockProducer is fenced.");
}
this.closed = true;
}
@@ -321,12 +328,18 @@ public boolean closed() {
return this.closed;
}
- public void fenceProducer() {
+ public synchronized void fenceProducer() {
verifyProducerState();
verifyTransactionsInitialized();
this.producerFenced = true;
}
+ public void fenceProducerOnClose() {
+ verifyProducerState();
+ verifyTransactionsInitialized();
+ this.producerFencedOnClose = true;
+ }
+
public boolean transactionInitialized() {
return this.transactionInitialized;
}
diff --git a/clients/src/main/java/org/apache/kafka/clients/producer/RecordMetadata.java b/clients/src/main/java/org/apache/kafka/clients/producer/RecordMetadata.java
index 0924244829b6f..1ae5cf654b8de 100644
--- a/clients/src/main/java/org/apache/kafka/clients/producer/RecordMetadata.java
+++ b/clients/src/main/java/org/apache/kafka/clients/producer/RecordMetadata.java
@@ -56,16 +56,6 @@ public RecordMetadata(TopicPartition topicPartition, long baseOffset, long relat
this.topicPartition = topicPartition;
}
- /**
- * @deprecated As of 0.11.0. Use @{@link RecordMetadata#RecordMetadata(TopicPartition, long, long, long, Long, int, int)}.
- */
- @Deprecated
- public RecordMetadata(TopicPartition topicPartition, long baseOffset, long relativeOffset, long timestamp,
- long checksum, int serializedKeySize, int serializedValueSize) {
- this(topicPartition, baseOffset, relativeOffset, timestamp, Long.valueOf(checksum), serializedKeySize,
- serializedValueSize);
- }
-
/**
* Indicates whether the record metadata includes the offset.
* @return true if the offset is included in the metadata, false otherwise.
diff --git a/clients/src/main/java/org/apache/kafka/clients/producer/internals/RecordAccumulator.java b/clients/src/main/java/org/apache/kafka/clients/producer/internals/RecordAccumulator.java
index e2b58448dc92e..31c6d754c9d92 100644
--- a/clients/src/main/java/org/apache/kafka/clients/producer/internals/RecordAccumulator.java
+++ b/clients/src/main/java/org/apache/kafka/clients/producer/internals/RecordAccumulator.java
@@ -19,6 +19,7 @@
import org.apache.kafka.clients.ApiVersions;
import org.apache.kafka.clients.producer.Callback;
import org.apache.kafka.common.Cluster;
+import org.apache.kafka.common.KafkaException;
import org.apache.kafka.common.MetricName;
import org.apache.kafka.common.Node;
import org.apache.kafka.common.PartitionInfo;
@@ -195,7 +196,7 @@ public RecordAppendResult append(TopicPartition tp,
Deque dq = getOrCreateDeque(tp);
synchronized (dq) {
if (closed)
- throw new IllegalStateException("Cannot send after the producer is closed.");
+ throw new KafkaException("Producer closed while send in progress");
RecordAppendResult appendResult = tryAppend(timestamp, key, value, headers, callback, dq);
if (appendResult != null)
return appendResult;
@@ -209,7 +210,7 @@ public RecordAppendResult append(TopicPartition tp,
synchronized (dq) {
// Need to check if producer is closed again after grabbing the dequeue lock.
if (closed)
- throw new IllegalStateException("Cannot send after the producer is closed.");
+ throw new KafkaException("Producer closed while send in progress");
RecordAppendResult appendResult = tryAppend(timestamp, key, value, headers, callback, dq);
if (appendResult != null) {
@@ -700,7 +701,7 @@ public void abortIncompleteBatches() {
* Go through incomplete batches and abort them.
*/
private void abortBatches() {
- abortBatches(new IllegalStateException("Producer is closed forcefully."));
+ abortBatches(new KafkaException("Producer is closed forcefully."));
}
/**
diff --git a/clients/src/main/java/org/apache/kafka/clients/producer/internals/Sender.java b/clients/src/main/java/org/apache/kafka/clients/producer/internals/Sender.java
index 7c941796ce453..a0fcad071ffe0 100644
--- a/clients/src/main/java/org/apache/kafka/clients/producer/internals/Sender.java
+++ b/clients/src/main/java/org/apache/kafka/clients/producer/internals/Sender.java
@@ -109,7 +109,7 @@ public class Sender implements Runnable {
private final SenderMetrics sensors;
/* the max time to wait for the server to respond to the request*/
- private final int requestTimeout;
+ private final int requestTimeoutMs;
/* The max time to wait before retrying a request which has failed */
private final long retryBackoffMs;
@@ -130,7 +130,7 @@ public Sender(LogContext logContext,
int retries,
SenderMetricsRegistry metricsRegistry,
Time time,
- int requestTimeout,
+ int requestTimeoutMs,
long retryBackoffMs,
TransactionManager transactionManager,
ApiVersions apiVersions) {
@@ -145,7 +145,7 @@ public Sender(LogContext logContext,
this.retries = retries;
this.time = time;
this.sensors = new SenderMetrics(metricsRegistry);
- this.requestTimeout = requestTimeout;
+ this.requestTimeoutMs = requestTimeoutMs;
this.retryBackoffMs = retryBackoffMs;
this.apiVersions = apiVersions;
this.transactionManager = transactionManager;
@@ -280,7 +280,7 @@ private long sendProducerData(long now) {
}
}
- List expiredBatches = this.accumulator.expiredBatches(this.requestTimeout, now);
+ List expiredBatches = this.accumulator.expiredBatches(this.requestTimeoutMs, now);
// Reset the producer id if an expired batch has previously been sent to the broker. Also update the metrics
// for expired batches. see the documentation of @TransactionState.resetProducerId to understand why
// we need to reset the producer id here.
@@ -342,12 +342,12 @@ private boolean maybeSendTransactionalRequest(long now) {
break;
}
- if (!NetworkClientUtils.awaitReady(client, targetNode, time, requestTimeout)) {
+ if (!NetworkClientUtils.awaitReady(client, targetNode, time, requestTimeoutMs)) {
transactionManager.lookupCoordinator(nextRequestHandler);
break;
}
} else {
- targetNode = awaitLeastLoadedNodeReady(requestTimeout);
+ targetNode = awaitLeastLoadedNodeReady(requestTimeoutMs);
}
if (targetNode != null) {
@@ -355,7 +355,7 @@ private boolean maybeSendTransactionalRequest(long now) {
time.sleep(nextRequestHandler.retryBackoffMs());
ClientRequest clientRequest = client.newClientRequest(targetNode.idString(),
- requestBuilder, now, true, nextRequestHandler);
+ requestBuilder, now, true, requestTimeoutMs, nextRequestHandler);
transactionManager.setInFlightTransactionalRequestCorrelationId(clientRequest.correlationId());
log.debug("Sending transactional request {} to node {}", requestBuilder, targetNode);
@@ -409,7 +409,7 @@ public void forceClose() {
private ClientResponse sendAndAwaitInitProducerIdRequest(Node node) throws IOException {
String nodeId = node.idString();
InitProducerIdRequest.Builder builder = new InitProducerIdRequest.Builder(null);
- ClientRequest request = client.newClientRequest(nodeId, builder, time.milliseconds(), true, null);
+ ClientRequest request = client.newClientRequest(nodeId, builder, time.milliseconds(), true, requestTimeoutMs, null);
return NetworkClientUtils.sendAndReceive(client, request, time);
}
@@ -422,9 +422,9 @@ private Node awaitLeastLoadedNodeReady(long remainingTimeMs) throws IOException
}
private void maybeWaitForProducerId() {
- while (!transactionManager.hasProducerId() && !transactionManager.hasError()) {
+ while (!forceClose && !transactionManager.hasProducerId() && !transactionManager.hasError()) {
try {
- Node node = awaitLeastLoadedNodeReady(requestTimeout);
+ Node node = awaitLeastLoadedNodeReady(requestTimeoutMs);
if (node != null) {
ClientResponse response = sendAndAwaitInitProducerIdRequest(node);
InitProducerIdResponse initProducerIdResponse = (InitProducerIdResponse) response.responseBody();
@@ -652,7 +652,7 @@ private boolean canRetry(ProducerBatch batch, ProduceResponse.PartitionResponse
*/
private void sendProduceRequests(Map> collated, long now) {
for (Map.Entry> entry : collated.entrySet())
- sendProduceRequest(now, entry.getKey(), acks, requestTimeout, entry.getValue());
+ sendProduceRequest(now, entry.getKey(), acks, requestTimeoutMs, entry.getValue());
}
/**
@@ -702,7 +702,8 @@ public void onComplete(ClientResponse response) {
};
String nodeId = Integer.toString(destination);
- ClientRequest clientRequest = client.newClientRequest(nodeId, requestBuilder, now, acks != 0, callback);
+ ClientRequest clientRequest = client.newClientRequest(nodeId, requestBuilder, now, acks != 0,
+ requestTimeoutMs, callback);
client.send(clientRequest, now);
log.trace("Sent produce request to {}: {}", nodeId, requestBuilder);
}
diff --git a/clients/src/main/java/org/apache/kafka/clients/producer/internals/TransactionManager.java b/clients/src/main/java/org/apache/kafka/clients/producer/internals/TransactionManager.java
index b242d5a65a639..124b5f8c68b37 100644
--- a/clients/src/main/java/org/apache/kafka/clients/producer/internals/TransactionManager.java
+++ b/clients/src/main/java/org/apache/kafka/clients/producer/internals/TransactionManager.java
@@ -26,6 +26,7 @@
import org.apache.kafka.common.errors.GroupAuthorizationException;
import org.apache.kafka.common.errors.TopicAuthorizationException;
import org.apache.kafka.common.protocol.Errors;
+import org.apache.kafka.common.record.DefaultRecordBatch;
import org.apache.kafka.common.record.RecordBatch;
import org.apache.kafka.common.requests.AbstractRequest;
import org.apache.kafka.common.requests.AbstractResponse;
@@ -419,7 +420,7 @@ synchronized void incrementSequenceNumber(TopicPartition topicPartition, int inc
if (currentSequenceNumber == null)
throw new IllegalStateException("Attempt to increment sequence number for a partition with no current sequence.");
- currentSequenceNumber += increment;
+ currentSequenceNumber = DefaultRecordBatch.incrementSequence(currentSequenceNumber, increment);
nextSequence.put(topicPartition, currentSequenceNumber);
}
@@ -1272,50 +1273,49 @@ String coordinatorKey() {
public void handleResponse(AbstractResponse response) {
TxnOffsetCommitResponse txnOffsetCommitResponse = (TxnOffsetCommitResponse) response;
boolean coordinatorReloaded = false;
- boolean hadFailure = false;
Map errors = txnOffsetCommitResponse.errors();
+ log.debug("Received TxnOffsetCommit response for consumer group {}: {}", builder.consumerGroupId(),
+ errors);
+
for (Map.Entry entry : errors.entrySet()) {
TopicPartition topicPartition = entry.getKey();
Errors error = entry.getValue();
if (error == Errors.NONE) {
- log.debug("Successfully added offsets {} from consumer group {} to transaction.",
- builder.offsets(), builder.consumerGroupId());
pendingTxnOffsetCommits.remove(topicPartition);
} else if (error == Errors.COORDINATOR_NOT_AVAILABLE
|| error == Errors.NOT_COORDINATOR
|| error == Errors.REQUEST_TIMED_OUT) {
- hadFailure = true;
if (!coordinatorReloaded) {
coordinatorReloaded = true;
lookupCoordinator(FindCoordinatorRequest.CoordinatorType.GROUP, builder.consumerGroupId());
}
- } else if (error == Errors.UNKNOWN_TOPIC_OR_PARTITION) {
- hadFailure = true;
+ } else if (error == Errors.UNKNOWN_TOPIC_OR_PARTITION
+ || error == Errors.COORDINATOR_LOAD_IN_PROGRESS) {
+ // If the topic is unknown or the coordinator is loading, retry with the current coordinator
+ continue;
} else if (error == Errors.GROUP_AUTHORIZATION_FAILED) {
abortableError(new GroupAuthorizationException(builder.consumerGroupId()));
- return;
+ break;
} else if (error == Errors.TRANSACTIONAL_ID_AUTHORIZATION_FAILED
|| error == Errors.INVALID_PRODUCER_EPOCH
|| error == Errors.UNSUPPORTED_FOR_MESSAGE_FORMAT) {
fatalError(error.exception());
- return;
+ break;
} else {
fatalError(new KafkaException("Unexpected error in TxnOffsetCommitResponse: " + error.message()));
- return;
+ break;
}
}
- if (!hadFailure || !result.isSuccessful()) {
- // all attempted partitions were either successful, or there was a fatal failure.
- // either way, we are not retrying, so complete the request.
+ if (result.isCompleted()) {
+ pendingTxnOffsetCommits.clear();
+ } else if (pendingTxnOffsetCommits.isEmpty()) {
result.done();
- return;
- }
-
- // retry the commits which failed with a retriable error.
- if (!pendingTxnOffsetCommits.isEmpty())
+ } else {
+ // Retry the commits which failed with a retriable error
reenqueue();
+ }
}
}
}
diff --git a/clients/src/main/java/org/apache/kafka/common/acl/AclBinding.java b/clients/src/main/java/org/apache/kafka/common/acl/AclBinding.java
index d264ef1a4f627..67dbfc0545c3b 100644
--- a/clients/src/main/java/org/apache/kafka/common/acl/AclBinding.java
+++ b/clients/src/main/java/org/apache/kafka/common/acl/AclBinding.java
@@ -18,49 +18,61 @@
package org.apache.kafka.common.acl;
import org.apache.kafka.common.annotation.InterfaceStability;
+import org.apache.kafka.common.resource.PatternType;
import org.apache.kafka.common.resource.Resource;
+import org.apache.kafka.common.resource.ResourcePattern;
import java.util.Objects;
/**
- * Represents a binding between a resource and an access control entry.
+ * Represents a binding between a resource pattern and an access control entry.
*
* The API for this class is still evolving and we may break compatibility in minor releases, if necessary.
*/
@InterfaceStability.Evolving
public class AclBinding {
- private final Resource resource;
+ private final ResourcePattern pattern;
private final AccessControlEntry entry;
+ /**
+ * Create an instance of this class with the provided parameters.
+ *
+ * @param pattern non-null resource pattern.
+ * @param entry non-null entry
+ */
+ public AclBinding(ResourcePattern pattern, AccessControlEntry entry) {
+ this.pattern = Objects.requireNonNull(pattern, "pattern");
+ this.entry = Objects.requireNonNull(entry, "entry");
+ }
+
/**
* Create an instance of this class with the provided parameters.
*
* @param resource non-null resource
* @param entry non-null entry
+ * @deprecated Since 2.0. Use {@link #AclBinding(ResourcePattern, AccessControlEntry)}
*/
+ @Deprecated
public AclBinding(Resource resource, AccessControlEntry entry) {
- Objects.requireNonNull(resource);
- this.resource = resource;
- Objects.requireNonNull(entry);
- this.entry = entry;
+ this(new ResourcePattern(resource.resourceType(), resource.name(), PatternType.LITERAL), entry);
}
/**
- * Return true if this binding has any UNKNOWN components.
+ * @return true if this binding has any UNKNOWN components.
*/
public boolean isUnknown() {
- return resource.isUnknown() || entry.isUnknown();
+ return pattern.isUnknown() || entry.isUnknown();
}
/**
- * Return the resource for this binding.
+ * @return the resource pattern for this binding.
*/
- public Resource resource() {
- return resource;
+ public ResourcePattern pattern() {
+ return pattern;
}
/**
- * Return the access control entry for this binding.
+ * @return the access control entry for this binding.
*/
public final AccessControlEntry entry() {
return entry;
@@ -70,24 +82,25 @@ public final AccessControlEntry entry() {
* Create a filter which matches only this AclBinding.
*/
public AclBindingFilter toFilter() {
- return new AclBindingFilter(resource.toFilter(), entry.toFilter());
+ return new AclBindingFilter(pattern.toFilter(), entry.toFilter());
}
@Override
public String toString() {
- return "(resource=" + resource + ", entry=" + entry + ")";
+ return "(pattern=" + pattern + ", entry=" + entry + ")";
}
@Override
public boolean equals(Object o) {
- if (!(o instanceof AclBinding))
- return false;
- AclBinding other = (AclBinding) o;
- return resource.equals(other.resource) && entry.equals(other.entry);
+ if (this == o) return true;
+ if (o == null || getClass() != o.getClass()) return false;
+ AclBinding that = (AclBinding) o;
+ return Objects.equals(pattern, that.pattern) &&
+ Objects.equals(entry, that.entry);
}
@Override
public int hashCode() {
- return Objects.hash(resource, entry);
+ return Objects.hash(pattern, entry);
}
}
diff --git a/clients/src/main/java/org/apache/kafka/common/acl/AclBindingFilter.java b/clients/src/main/java/org/apache/kafka/common/acl/AclBindingFilter.java
index 5841b5aeb9a0e..3168ec61cfcef 100644
--- a/clients/src/main/java/org/apache/kafka/common/acl/AclBindingFilter.java
+++ b/clients/src/main/java/org/apache/kafka/common/acl/AclBindingFilter.java
@@ -18,7 +18,9 @@
package org.apache.kafka.common.acl;
import org.apache.kafka.common.annotation.InterfaceStability;
+import org.apache.kafka.common.resource.PatternType;
import org.apache.kafka.common.resource.ResourceFilter;
+import org.apache.kafka.common.resource.ResourcePatternFilter;
import java.util.Objects;
@@ -29,43 +31,53 @@
*/
@InterfaceStability.Evolving
public class AclBindingFilter {
- private final ResourceFilter resourceFilter;
+ private final ResourcePatternFilter patternFilter;
private final AccessControlEntryFilter entryFilter;
/**
* A filter which matches any ACL binding.
*/
- public static final AclBindingFilter ANY = new AclBindingFilter(ResourceFilter.ANY, AccessControlEntryFilter.ANY);
+ public static final AclBindingFilter ANY = new AclBindingFilter(ResourcePatternFilter.ANY, AccessControlEntryFilter.ANY);
+
+ /**
+ * Create an instance of this filter with the provided parameters.
+ *
+ * @param patternFilter non-null pattern filter
+ * @param entryFilter non-null access control entry filter
+ */
+ public AclBindingFilter(ResourcePatternFilter patternFilter, AccessControlEntryFilter entryFilter) {
+ this.patternFilter = Objects.requireNonNull(patternFilter, "patternFilter");
+ this.entryFilter = Objects.requireNonNull(entryFilter, "entryFilter");
+ }
/**
* Create an instance of this filter with the provided parameters.
*
* @param resourceFilter non-null resource filter
* @param entryFilter non-null access control entry filter
+ * @deprecated Since 2.0. Use {@link #AclBindingFilter(ResourcePatternFilter, AccessControlEntryFilter)}
*/
+ @Deprecated
public AclBindingFilter(ResourceFilter resourceFilter, AccessControlEntryFilter entryFilter) {
- Objects.requireNonNull(resourceFilter);
- this.resourceFilter = resourceFilter;
- Objects.requireNonNull(entryFilter);
- this.entryFilter = entryFilter;
+ this(new ResourcePatternFilter(resourceFilter.resourceType(), resourceFilter.name(), PatternType.LITERAL), entryFilter);
}
/**
- * Return true if this filter has any UNKNOWN components.
+ * @return {@code true} if this filter has any UNKNOWN components.
*/
public boolean isUnknown() {
- return resourceFilter.isUnknown() || entryFilter.isUnknown();
+ return patternFilter.isUnknown() || entryFilter.isUnknown();
}
/**
- * Return the resource filter.
+ * @return the resource pattern filter.
*/
- public ResourceFilter resourceFilter() {
- return resourceFilter;
+ public ResourcePatternFilter patternFilter() {
+ return patternFilter;
}
/**
- * Return the access control entry filter.
+ * @return the access control entry filter.
*/
public final AccessControlEntryFilter entryFilter() {
return entryFilter;
@@ -73,15 +85,16 @@ public final AccessControlEntryFilter entryFilter() {
@Override
public String toString() {
- return "(resourceFilter=" + resourceFilter + ", entryFilter=" + entryFilter + ")";
+ return "(patternFilter=" + patternFilter + ", entryFilter=" + entryFilter + ")";
}
@Override
public boolean equals(Object o) {
- if (!(o instanceof AclBindingFilter))
- return false;
- AclBindingFilter other = (AclBindingFilter) o;
- return resourceFilter.equals(other.resourceFilter) && entryFilter.equals(other.entryFilter);
+ if (this == o) return true;
+ if (o == null || getClass() != o.getClass()) return false;
+ AclBindingFilter that = (AclBindingFilter) o;
+ return Objects.equals(patternFilter, that.patternFilter) &&
+ Objects.equals(entryFilter, that.entryFilter);
}
/**
@@ -89,14 +102,14 @@ public boolean equals(Object o) {
* there are no ANY or UNKNOWN fields.
*/
public boolean matchesAtMostOne() {
- return resourceFilter.matchesAtMostOne() && entryFilter.matchesAtMostOne();
+ return patternFilter.matchesAtMostOne() && entryFilter.matchesAtMostOne();
}
/**
* Return a string describing an ANY or UNKNOWN field, or null if there is no such field.
*/
public String findIndefiniteField() {
- String indefinite = resourceFilter.findIndefiniteField();
+ String indefinite = patternFilter.findIndefiniteField();
if (indefinite != null)
return indefinite;
return entryFilter.findIndefiniteField();
@@ -106,11 +119,11 @@ public String findIndefiniteField() {
* Return true if the resource filter matches the binding's resource and the entry filter matches binding's entry.
*/
public boolean matches(AclBinding binding) {
- return resourceFilter.matches(binding.resource()) && entryFilter.matches(binding.entry());
+ return patternFilter.matches(binding.pattern()) && entryFilter.matches(binding.entry());
}
@Override
public int hashCode() {
- return Objects.hash(resourceFilter, entryFilter);
+ return Objects.hash(patternFilter, entryFilter);
}
}
diff --git a/clients/src/main/java/org/apache/kafka/common/config/ConfigChangeCallback.java b/clients/src/main/java/org/apache/kafka/common/config/ConfigChangeCallback.java
index d4c9948bc9308..faa7d3d87074a 100644
--- a/clients/src/main/java/org/apache/kafka/common/config/ConfigChangeCallback.java
+++ b/clients/src/main/java/org/apache/kafka/common/config/ConfigChangeCallback.java
@@ -16,6 +16,8 @@
*/
package org.apache.kafka.common.config;
+import org.apache.kafka.common.config.provider.ConfigProvider;
+
/**
* A callback passed to {@link ConfigProvider} for subscribing to changes.
*/
diff --git a/clients/src/main/java/org/apache/kafka/common/config/ConfigData.java b/clients/src/main/java/org/apache/kafka/common/config/ConfigData.java
index 2bd0ff6b06ab3..8661ee16cbab1 100644
--- a/clients/src/main/java/org/apache/kafka/common/config/ConfigData.java
+++ b/clients/src/main/java/org/apache/kafka/common/config/ConfigData.java
@@ -16,6 +16,8 @@
*/
package org.apache.kafka.common.config;
+import org.apache.kafka.common.config.provider.ConfigProvider;
+
import java.util.Map;
/**
diff --git a/clients/src/main/java/org/apache/kafka/common/config/ConfigDef.java b/clients/src/main/java/org/apache/kafka/common/config/ConfigDef.java
index c9efb82e061c5..064e08584c7ed 100644
--- a/clients/src/main/java/org/apache/kafka/common/config/ConfigDef.java
+++ b/clients/src/main/java/org/apache/kafka/common/config/ConfigDef.java
@@ -711,9 +711,16 @@ else if (value instanceof String)
case CLASS:
if (value instanceof Class)
return value;
- else if (value instanceof String)
- return Class.forName(trimmed, true, Utils.getContextOrKafkaClassLoader());
- else
+ else if (value instanceof String) {
+ ClassLoader contextOrKafkaClassLoader = Utils.getContextOrKafkaClassLoader();
+ // Use loadClass here instead of Class.forName because the name we use here may be an alias
+ // and not match the name of the class that gets loaded. If that happens, Class.forName can
+ // throw an exception.
+ Class> klass = contextOrKafkaClassLoader.loadClass(trimmed);
+ // Invoke forName here with the true name of the requested class to cause class
+ // initialization to take place.
+ return Class.forName(klass.getName(), true, contextOrKafkaClassLoader);
+ } else
throw new ConfigException(name, value, "Expected a Class instance or class name.");
default:
throw new IllegalStateException("Unknown type.");
diff --git a/clients/src/main/java/org/apache/kafka/common/config/ConfigResource.java b/clients/src/main/java/org/apache/kafka/common/config/ConfigResource.java
index da718f54d547e..5343a6bcdfd3d 100644
--- a/clients/src/main/java/org/apache/kafka/common/config/ConfigResource.java
+++ b/clients/src/main/java/org/apache/kafka/common/config/ConfigResource.java
@@ -17,7 +17,12 @@
package org.apache.kafka.common.config;
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.Map;
import java.util.Objects;
+import java.util.function.Function;
+import java.util.stream.Collectors;
/**
* A class representing resources that have configs.
@@ -28,7 +33,25 @@ public final class ConfigResource {
* Type of resource.
*/
public enum Type {
- BROKER, TOPIC, UNKNOWN;
+ BROKER((byte) 4), TOPIC((byte) 2), UNKNOWN((byte) 0);
+
+ private static final Map TYPES = Collections.unmodifiableMap(
+ Arrays.stream(values()).collect(Collectors.toMap(Type::id, Function.identity()))
+ );
+
+ private final byte id;
+
+ Type(final byte id) {
+ this.id = id;
+ }
+
+ public byte id() {
+ return id;
+ }
+
+ public static Type forId(final byte id) {
+ return TYPES.getOrDefault(id, UNKNOWN);
+ }
}
private final Type type;
diff --git a/clients/src/main/java/org/apache/kafka/common/config/ConfigTransformer.java b/clients/src/main/java/org/apache/kafka/common/config/ConfigTransformer.java
index 7c3c516b07341..c1264724d904b 100644
--- a/clients/src/main/java/org/apache/kafka/common/config/ConfigTransformer.java
+++ b/clients/src/main/java/org/apache/kafka/common/config/ConfigTransformer.java
@@ -16,6 +16,9 @@
*/
package org.apache.kafka.common.config;
+import org.apache.kafka.common.config.provider.ConfigProvider;
+import org.apache.kafka.common.config.provider.FileConfigProvider;
+
import java.util.ArrayList;
import java.util.HashMap;
import java.util.HashSet;
@@ -50,7 +53,7 @@
* {@link ConfigProvider#unsubscribe(String, Set, ConfigChangeCallback)} methods.
*/
public class ConfigTransformer {
- private static final Pattern DEFAULT_PATTERN = Pattern.compile("\\$\\{(.*?):((.*?):)?(.*?)\\}");
+ public static final Pattern DEFAULT_PATTERN = Pattern.compile("\\$\\{([^}]*?):(([^}]*?):)?([^}]*?)\\}");
private static final String EMPTY_PATH = "";
private final Map configProviders;
@@ -77,11 +80,13 @@ public ConfigTransformerResult transform(Map configs) {
// Collect the variables from the given configs that need transformation
for (Map.Entry config : configs.entrySet()) {
- List vars = getVars(config.getKey(), config.getValue(), DEFAULT_PATTERN);
- for (ConfigVariable var : vars) {
- Map> keysByPath = keysByProvider.computeIfAbsent(var.providerName, k -> new HashMap<>());
- Set keys = keysByPath.computeIfAbsent(var.path, k -> new HashSet<>());
- keys.add(var.variable);
+ if (config.getValue() != null) {
+ List vars = getVars(config.getKey(), config.getValue(), DEFAULT_PATTERN);
+ for (ConfigVariable var : vars) {
+ Map> keysByPath = keysByProvider.computeIfAbsent(var.providerName, k -> new HashMap<>());
+ Set keys = keysByPath.computeIfAbsent(var.path, k -> new HashSet<>());
+ keys.add(var.variable);
+ }
}
}
@@ -128,6 +133,9 @@ private static List getVars(String key, String value, Pattern pa
private static String replace(Map>> lookupsByProvider,
String value,
Pattern pattern) {
+ if (value == null) {
+ return null;
+ }
Matcher matcher = pattern.matcher(value);
StringBuilder builder = new StringBuilder();
int i = 0;
diff --git a/clients/src/main/java/org/apache/kafka/common/config/ConfigTransformerResult.java b/clients/src/main/java/org/apache/kafka/common/config/ConfigTransformerResult.java
index df7bea62f3759..a05669cb6c3ee 100644
--- a/clients/src/main/java/org/apache/kafka/common/config/ConfigTransformerResult.java
+++ b/clients/src/main/java/org/apache/kafka/common/config/ConfigTransformerResult.java
@@ -16,6 +16,8 @@
*/
package org.apache.kafka.common.config;
+import org.apache.kafka.common.config.provider.ConfigProvider;
+
import java.util.Map;
/**
diff --git a/clients/src/main/java/org/apache/kafka/common/config/TopicConfig.java b/clients/src/main/java/org/apache/kafka/common/config/TopicConfig.java
index d6b70032626ad..fb2208c0328d1 100755
--- a/clients/src/main/java/org/apache/kafka/common/config/TopicConfig.java
+++ b/clients/src/main/java/org/apache/kafka/common/config/TopicConfig.java
@@ -165,4 +165,11 @@ public class TopicConfig {
"the timestamp when a broker receives a message and the timestamp specified in the message. If " +
"message.timestamp.type=CreateTime, a message will be rejected if the difference in timestamp " +
"exceeds this threshold. This configuration is ignored if message.timestamp.type=LogAppendTime.";
+
+ public static final String MESSAGE_DOWNCONVERSION_ENABLE_CONFIG = "message.downconversion.enable";
+ public static final String MESSAGE_DOWNCONVERSION_ENABLE_DOC = "This configuration controls whether " +
+ "down-conversion of message formats is enabled to satisfy consume requests. When set to false, " +
+ "broker will not perform down-conversion for consumers expecting an older message format. The broker responds " +
+ "with UNSUPPORTED_VERSION error for consume requests from such older clients. This configuration" +
+ "does not apply to any message format conversion that might be required for replication to followers.";
}
diff --git a/clients/src/main/java/org/apache/kafka/common/config/ConfigProvider.java b/clients/src/main/java/org/apache/kafka/common/config/provider/ConfigProvider.java
similarity index 94%
rename from clients/src/main/java/org/apache/kafka/common/config/ConfigProvider.java
rename to clients/src/main/java/org/apache/kafka/common/config/provider/ConfigProvider.java
index 7133baaebd01a..8561511e64e30 100644
--- a/clients/src/main/java/org/apache/kafka/common/config/ConfigProvider.java
+++ b/clients/src/main/java/org/apache/kafka/common/config/provider/ConfigProvider.java
@@ -14,9 +14,11 @@
* See the License for the specific language governing permissions and
* limitations under the License.
*/
-package org.apache.kafka.common.config;
+package org.apache.kafka.common.config.provider;
import org.apache.kafka.common.Configurable;
+import org.apache.kafka.common.config.ConfigChangeCallback;
+import org.apache.kafka.common.config.ConfigData;
import java.io.Closeable;
import java.util.Set;
diff --git a/clients/src/main/java/org/apache/kafka/common/config/FileConfigProvider.java b/clients/src/main/java/org/apache/kafka/common/config/provider/FileConfigProvider.java
similarity index 92%
rename from clients/src/main/java/org/apache/kafka/common/config/FileConfigProvider.java
rename to clients/src/main/java/org/apache/kafka/common/config/provider/FileConfigProvider.java
index fefc93566f389..d25183a80119a 100644
--- a/clients/src/main/java/org/apache/kafka/common/config/FileConfigProvider.java
+++ b/clients/src/main/java/org/apache/kafka/common/config/provider/FileConfigProvider.java
@@ -14,8 +14,12 @@
* See the License for the specific language governing permissions and
* limitations under the License.
*/
-package org.apache.kafka.common.config;
+package org.apache.kafka.common.config.provider;
+import org.apache.kafka.common.config.ConfigData;
+import org.apache.kafka.common.config.ConfigException;
+
+import java.io.BufferedReader;
import java.io.FileInputStream;
import java.io.IOException;
import java.io.InputStreamReader;
@@ -93,7 +97,7 @@ public ConfigData get(String path, Set keys) {
// visible for testing
protected Reader reader(String path) throws IOException {
- return new InputStreamReader(new FileInputStream(path), StandardCharsets.UTF_8);
+ return new BufferedReader(new InputStreamReader(new FileInputStream(path), StandardCharsets.UTF_8));
}
public void close() {
diff --git a/clients/src/main/java/org/apache/kafka/common/errors/GroupCoordinatorNotAvailableException.java b/clients/src/main/java/org/apache/kafka/common/errors/GroupCoordinatorNotAvailableException.java
deleted file mode 100644
index 03a7719a6f428..0000000000000
--- a/clients/src/main/java/org/apache/kafka/common/errors/GroupCoordinatorNotAvailableException.java
+++ /dev/null
@@ -1,47 +0,0 @@
-/*
- * 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.common.errors;
-
-/**
- * The broker returns this error code for consumer metadata requests or offset commit requests if the offsets topic has
- * not yet been created.
- *
- * @deprecated As of Kafka 0.11, this has been replaced by {@link CoordinatorNotAvailableException}
- */
-@Deprecated
-public class GroupCoordinatorNotAvailableException extends RetriableException {
- public static final GroupCoordinatorNotAvailableException INSTANCE = new GroupCoordinatorNotAvailableException();
-
- private static final long serialVersionUID = 1L;
-
- public GroupCoordinatorNotAvailableException() {
- super();
- }
-
- public GroupCoordinatorNotAvailableException(String message) {
- super(message);
- }
-
- public GroupCoordinatorNotAvailableException(String message, Throwable cause) {
- super(message, cause);
- }
-
- public GroupCoordinatorNotAvailableException(Throwable cause) {
- super(cause);
- }
-
-}
diff --git a/clients/src/main/java/org/apache/kafka/common/errors/NotCoordinatorForGroupException.java b/clients/src/main/java/org/apache/kafka/common/errors/ListenerNotFoundException.java
similarity index 61%
rename from clients/src/main/java/org/apache/kafka/common/errors/NotCoordinatorForGroupException.java
rename to clients/src/main/java/org/apache/kafka/common/errors/ListenerNotFoundException.java
index cee649502e979..82c5d892f49ba 100644
--- a/clients/src/main/java/org/apache/kafka/common/errors/NotCoordinatorForGroupException.java
+++ b/clients/src/main/java/org/apache/kafka/common/errors/ListenerNotFoundException.java
@@ -17,30 +17,22 @@
package org.apache.kafka.common.errors;
/**
- * The broker returns this error code if it receives an offset fetch or commit request for a consumer group that it is
- * not a coordinator for.
- *
- * @deprecated As of Kafka 0.11, this has been replaced by {@link NotCoordinatorException}
+ * The leader does not have an endpoint corresponding to the listener on which metadata was requested.
+ * This could indicate a broker configuration error or a transient error when listeners are updated
+ * dynamically and client requests are processed before all brokers have updated their listeners.
+ * This is currently used only for missing listeners on leader brokers, but may be used for followers
+ * in future.
*/
-@Deprecated
-public class NotCoordinatorForGroupException extends RetriableException {
+public class ListenerNotFoundException extends InvalidMetadataException {
private static final long serialVersionUID = 1L;
- public NotCoordinatorForGroupException() {
- super();
- }
-
- public NotCoordinatorForGroupException(String message) {
+ public ListenerNotFoundException(String message) {
super(message);
}
- public NotCoordinatorForGroupException(String message, Throwable cause) {
+ public ListenerNotFoundException(String message, Throwable cause) {
super(message, cause);
}
- public NotCoordinatorForGroupException(Throwable cause) {
- super(cause);
- }
-
}
diff --git a/clients/src/main/java/org/apache/kafka/common/internals/PartitionStates.java b/clients/src/main/java/org/apache/kafka/common/internals/PartitionStates.java
index 605372c6ec7e9..5b904c2671771 100644
--- a/clients/src/main/java/org/apache/kafka/common/internals/PartitionStates.java
+++ b/clients/src/main/java/org/apache/kafka/common/internals/PartitionStates.java
@@ -36,11 +36,17 @@
* topic would "wrap around" and appear twice. However, as partitions are fetched in different orders and partition
* leadership changes, we will deviate from the optimal. If this turns out to be an issue in practice, we can improve
* it by tracking the partitions per node or calling `set` every so often.
+ *
+ * Note that this class is not thread-safe with the exception of {@link #size()} which returns the number of
+ * partitions currently tracked.
*/
public class PartitionStates {
private final LinkedHashMap map = new LinkedHashMap<>();
+ /* the number of partitions that are currently assigned available in a thread safe manner */
+ private volatile int size = 0;
+
public PartitionStates() {}
public void moveToEnd(TopicPartition topicPartition) {
@@ -52,10 +58,12 @@ public void moveToEnd(TopicPartition topicPartition) {
public void updateAndMoveToEnd(TopicPartition topicPartition, S state) {
map.remove(topicPartition);
map.put(topicPartition, state);
+ updateSize();
}
public void remove(TopicPartition topicPartition) {
map.remove(topicPartition);
+ updateSize();
}
/**
@@ -67,6 +75,7 @@ public Set partitionSet() {
public void clear() {
map.clear();
+ updateSize();
}
public boolean contains(TopicPartition topicPartition) {
@@ -95,8 +104,11 @@ public S stateValue(TopicPartition topicPartition) {
return map.get(topicPartition);
}
+ /**
+ * Get the number of partitions that are currently being tracked. This is thread-safe.
+ */
public int size() {
- return map.size();
+ return size;
}
/**
@@ -108,6 +120,11 @@ public int size() {
public void set(Map partitionToState) {
map.clear();
update(partitionToState);
+ updateSize();
+ }
+
+ private void updateSize() {
+ size = map.size();
}
private void update(Map partitionToState) {
diff --git a/clients/src/main/java/org/apache/kafka/common/metrics/Metrics.java b/clients/src/main/java/org/apache/kafka/common/metrics/Metrics.java
index a6da9f90397d2..9e2b6f18f0cfc 100644
--- a/clients/src/main/java/org/apache/kafka/common/metrics/Metrics.java
+++ b/clients/src/main/java/org/apache/kafka/common/metrics/Metrics.java
@@ -41,6 +41,8 @@
import java.util.concurrent.ThreadFactory;
import java.util.concurrent.TimeUnit;
+import static java.util.Collections.emptyList;
+
/**
* A registry of sensors and metrics.
*
@@ -446,6 +448,9 @@ public void removeSensor(String name) {
removeMetric(metric.metricName());
log.debug("Removed sensor with name {}", name);
childSensors = childrenSensors.remove(sensor);
+ for (final Sensor parent : sensor.parents()) {
+ childrenSensors.getOrDefault(parent, emptyList()).remove(sensor);
+ }
}
}
}
diff --git a/clients/src/main/java/org/apache/kafka/common/metrics/Sensor.java b/clients/src/main/java/org/apache/kafka/common/metrics/Sensor.java
index e4bf1aeee699d..1af9419bc757a 100644
--- a/clients/src/main/java/org/apache/kafka/common/metrics/Sensor.java
+++ b/clients/src/main/java/org/apache/kafka/common/metrics/Sensor.java
@@ -22,7 +22,6 @@
import org.apache.kafka.common.utils.Utils;
import java.util.ArrayList;
-import java.util.Collections;
import java.util.HashSet;
import java.util.LinkedHashMap;
import java.util.LinkedList;
@@ -32,6 +31,9 @@
import java.util.Set;
import java.util.concurrent.TimeUnit;
+import static java.util.Arrays.asList;
+import static java.util.Collections.unmodifiableList;
+
/**
* A sensor applies a continuous sequence of numerical values to a set of associated metrics. For example a sensor on
* message size would record a sequence of message sizes using the {@link #record(double)} api and would maintain a set
@@ -48,6 +50,7 @@ public final class Sensor {
private final Time time;
private volatile long lastRecordTime;
private final long inactiveSensorExpirationTimeMs;
+ private final Object metricLock;
public enum RecordingLevel {
INFO(0, "INFO"), DEBUG(1, "DEBUG");
@@ -113,6 +116,7 @@ public boolean shouldRecord(final int configId) {
this.inactiveSensorExpirationTimeMs = TimeUnit.MILLISECONDS.convert(inactiveSensorExpirationTimeSeconds, TimeUnit.SECONDS);
this.lastRecordTime = time.milliseconds();
this.recordingLevel = recordingLevel;
+ this.metricLock = new Object();
checkForest(new HashSet());
}
@@ -131,6 +135,10 @@ public String name() {
return this.name;
}
+ List parents() {
+ return unmodifiableList(asList(parents));
+ }
+
/**
* Record an occurrence, this is just short-hand for {@link #record(double) record(1.0)}
*/
@@ -174,9 +182,11 @@ public void record(double value, long timeMs, boolean checkQuotas) {
if (shouldRecord()) {
this.lastRecordTime = timeMs;
synchronized (this) {
- // increment all the stats
- for (Stat stat : this.stats)
- stat.record(config, value, timeMs);
+ synchronized (metricLock()) {
+ // increment all the stats
+ for (Stat stat : this.stats)
+ stat.record(config, value, timeMs);
+ }
if (checkQuotas)
checkQuotas(timeMs);
}
@@ -229,7 +239,7 @@ public synchronized boolean add(CompoundStat stat, MetricConfig config) {
return false;
this.stats.add(Utils.notNull(stat));
- Object lock = metricLock(stat);
+ Object lock = metricLock();
for (NamedMeasurable m : stat.stats()) {
final KafkaMetric metric = new KafkaMetric(lock, m.name(), m.stat(), config == null ? this.config : config, time);
if (!metrics.containsKey(metric.metricName())) {
@@ -265,7 +275,7 @@ public synchronized boolean add(final MetricName metricName, final MeasurableSta
return true;
} else {
final KafkaMetric metric = new KafkaMetric(
- metricLock(stat),
+ metricLock(),
Utils.notNull(metricName),
Utils.notNull(stat),
config == null ? this.config : config,
@@ -287,14 +297,30 @@ public boolean hasExpired() {
}
synchronized List metrics() {
- return Collections.unmodifiableList(new LinkedList<>(this.metrics.values()));
+ return unmodifiableList(new LinkedList<>(this.metrics.values()));
}
/**
- * KafkaMetrics of sensors which use SampledStat should be synchronized on the Sensor object
- * to allow concurrent reads and updates. For simplicity, all sensors are synchronized on Sensor.
+ * KafkaMetrics of sensors which use SampledStat should be synchronized on the same lock
+ * for sensor record and metric value read to allow concurrent reads and updates. For simplicity,
+ * all sensors are synchronized on this object.
+ *
+ * Sensor object is not used as a lock for reading metric value since metrics reporter is
+ * invoked while holding Sensor and Metrics locks to report addition and removal of metrics
+ * and synchronized reporters may deadlock if Sensor lock is used for reading metrics values.
+ * Note that Sensor object itself is used as a lock to protect the access to stats and metrics
+ * while recording metric values, adding and deleting sensors.
+ *
+ * Locking order (assume all MetricsReporter methods may be synchronized):
+ *
+ * - Sensor#add: Sensor -> Metrics -> MetricsReporter
+ * - Metrics#removeSensor: Sensor -> Metrics -> MetricsReporter
+ * - KafkaMetric#metricValue: MetricsReporter -> Sensor#metricLock
+ * - Sensor#record: Sensor -> Sensor#metricLock
+ *
+ *
*/
- private Object metricLock(Stat stat) {
- return this;
+ private Object metricLock() {
+ return metricLock;
}
}
diff --git a/clients/src/main/java/org/apache/kafka/common/metrics/stats/Meter.java b/clients/src/main/java/org/apache/kafka/common/metrics/stats/Meter.java
index 09263cecae89c..91d4461d2b52f 100644
--- a/clients/src/main/java/org/apache/kafka/common/metrics/stats/Meter.java
+++ b/clients/src/main/java/org/apache/kafka/common/metrics/stats/Meter.java
@@ -61,6 +61,9 @@ public Meter(SampledStat rateStat, MetricName rateMetricName, MetricName totalMe
* Construct a Meter with provided time unit and provided {@link SampledStat} stats for Rate
*/
public Meter(TimeUnit unit, SampledStat rateStat, MetricName rateMetricName, MetricName totalMetricName) {
+ if (!(rateStat instanceof SampledTotal) && !(rateStat instanceof Count)) {
+ throw new IllegalArgumentException("Meter is supported only for SampledTotal and Count");
+ }
this.total = new Total();
this.rate = new Rate(unit, rateStat);
this.rateMetricName = rateMetricName;
@@ -77,6 +80,8 @@ public List stats() {
@Override
public void record(MetricConfig config, double value, long timeMs) {
rate.record(config, value, timeMs);
- total.record(config, value, timeMs);
+ // Total metrics with Count stat should record 1.0 (as recorded in the count)
+ double totalValue = (rate.stat instanceof Count) ? 1.0 : value;
+ total.record(config, totalValue, timeMs);
}
}
diff --git a/clients/src/main/java/org/apache/kafka/common/network/NetworkReceive.java b/clients/src/main/java/org/apache/kafka/common/network/NetworkReceive.java
index 564fbcd8c217c..55354ac8d6417 100644
--- a/clients/src/main/java/org/apache/kafka/common/network/NetworkReceive.java
+++ b/clients/src/main/java/org/apache/kafka/common/network/NetworkReceive.java
@@ -19,7 +19,6 @@
import java.io.EOFException;
import java.io.IOException;
import java.nio.ByteBuffer;
-import java.nio.channels.ReadableByteChannel;
import java.nio.channels.ScatteringByteChannel;
import org.apache.kafka.common.memory.MemoryPool;
import org.slf4j.Logger;
@@ -90,33 +89,6 @@ public boolean complete() {
}
public long readFrom(ScatteringByteChannel channel) throws IOException {
- return readFromReadableChannel(channel);
- }
-
- @Override
- public boolean requiredMemoryAmountKnown() {
- return requestedBufferSize != -1;
- }
-
- @Override
- public boolean memoryAllocated() {
- return buffer != null;
- }
-
-
- @Override
- public void close() throws IOException {
- if (buffer != null && buffer != EMPTY_BUFFER) {
- memoryPool.release(buffer);
- buffer = null;
- }
- }
-
- // Need a method to read from ReadableByteChannel because BlockingChannel requires read with timeout
- // See: http://stackoverflow.com/questions/2866557/timeout-for-socketchannel-doesnt-work
- // This can go away after we get rid of BlockingChannel
- @Deprecated
- public long readFromReadableChannel(ReadableByteChannel channel) throws IOException {
int read = 0;
if (size.hasRemaining()) {
int bytesRead = channel.read(size);
@@ -151,8 +123,35 @@ public long readFromReadableChannel(ReadableByteChannel channel) throws IOExcept
return read;
}
+ @Override
+ public boolean requiredMemoryAmountKnown() {
+ return requestedBufferSize != -1;
+ }
+
+ @Override
+ public boolean memoryAllocated() {
+ return buffer != null;
+ }
+
+
+ @Override
+ public void close() throws IOException {
+ if (buffer != null && buffer != EMPTY_BUFFER) {
+ memoryPool.release(buffer);
+ buffer = null;
+ }
+ }
+
public ByteBuffer payload() {
return this.buffer;
}
+ /**
+ * Returns the total size of the receive including payload and size buffer
+ * for use in metrics. This is consistent with {@link NetworkSend#size()}
+ */
+ public int size() {
+ return payload().limit() + size.limit();
+ }
+
}
diff --git a/clients/src/main/java/org/apache/kafka/common/network/Selector.java b/clients/src/main/java/org/apache/kafka/common/network/Selector.java
index 334ca79f0350d..057b1ac928e03 100644
--- a/clients/src/main/java/org/apache/kafka/common/network/Selector.java
+++ b/clients/src/main/java/org/apache/kafka/common/network/Selector.java
@@ -31,6 +31,7 @@
import org.apache.kafka.common.metrics.stats.SampledStat;
import org.apache.kafka.common.utils.LogContext;
import org.apache.kafka.common.utils.Time;
+import org.apache.kafka.common.utils.Utils;
import org.slf4j.Logger;
import java.io.IOException;
@@ -53,6 +54,7 @@
import java.util.Map;
import java.util.Set;
import java.util.concurrent.TimeUnit;
+import java.util.concurrent.atomic.AtomicReference;
/**
* A nioSelector interface for doing non-blocking multi-connection network I/O.
@@ -265,6 +267,7 @@ private void configureSocketChannel(SocketChannel socketChannel, int sendBufferS
public void register(String id, SocketChannel socketChannel) throws IOException {
ensureNotRegistered(id);
registerChannel(id, socketChannel, SelectionKey.OP_READ);
+ this.sensors.connectionCreated.record();
}
private void ensureNotRegistered(String id) {
@@ -278,6 +281,8 @@ private SelectionKey registerChannel(String id, SocketChannel socketChannel, int
SelectionKey key = socketChannel.register(nioSelector, interestedOps);
KafkaChannel channel = buildAndAttachKafkaChannel(socketChannel, id, key);
this.channels.put(id, channel);
+ if (idleExpiryManager != null)
+ idleExpiryManager.update(channel.id(), time.nanoseconds());
return key;
}
@@ -310,15 +315,24 @@ public void wakeup() {
@Override
public void close() {
List connections = new ArrayList<>(channels.keySet());
- for (String id : connections)
- close(id);
try {
- this.nioSelector.close();
- } catch (IOException | SecurityException e) {
- log.error("Exception closing nioSelector:", e);
+ for (String id : connections)
+ close(id);
+ } finally {
+ // If there is any exception thrown in close(id), we should still be able
+ // to close the remaining objects, especially the sensors because keeping
+ // the sensors may lead to failure to start up the ReplicaFetcherThread if
+ // the old sensors with the same names has not yet been cleaned up.
+ AtomicReference firstException = new AtomicReference<>();
+ Utils.closeQuietly(nioSelector, "nioSelector", firstException);
+ Utils.closeQuietly(sensors, "sensors", firstException);
+ Utils.closeQuietly(channelBuilder, "channelBuilder", firstException);
+ Throwable exception = firstException.get();
+ if (exception instanceof RuntimeException && !(exception instanceof SecurityException)) {
+ throw (RuntimeException) exception;
+ }
+
}
- sensors.close();
- channelBuilder.close();
}
/**
@@ -498,7 +512,9 @@ void pollSelectionKeys(Set selectionKeys,
//this channel has bytes enqueued in intermediary buffers that we could not read
//(possibly because no memory). it may be the case that the underlying socket will
//not come up in the next poll() and so we need to remember this channel for the
- //next poll call otherwise data may be stuck in said buffers forever.
+ //next poll call otherwise data may be stuck in said buffers forever. If we attempt
+ //to process buffered data and no progress is made, the channel buffered status is
+ //cleared to avoid the overhead of checking every time.
keysWithBufferedRead.add(key);
}
@@ -859,7 +875,7 @@ private void addToCompletedReceives() {
private void addToCompletedReceives(KafkaChannel channel, Deque stagedDeque) {
NetworkReceive networkReceive = stagedDeque.poll();
this.completedReceives.add(networkReceive);
- this.sensors.recordBytesReceived(channel.id(), networkReceive.payload().limit());
+ this.sensors.recordBytesReceived(channel.id(), networkReceive.size());
}
// only for testing
@@ -873,7 +889,7 @@ public int numStagedReceives(KafkaChannel channel) {
return deque == null ? 0 : deque.size();
}
- private class SelectorMetrics {
+ private class SelectorMetrics implements AutoCloseable {
private final Metrics metrics;
private final String metricGrpPrefix;
private final Map metricTags;
@@ -1053,6 +1069,7 @@ public void recordBytesReceived(String connection, int bytes) {
}
}
+ @Override
public void close() {
for (MetricName metricName : topLevelMetricNames)
metrics.removeMetric(metricName);
diff --git a/clients/src/main/java/org/apache/kafka/common/network/SslTransportLayer.java b/clients/src/main/java/org/apache/kafka/common/network/SslTransportLayer.java
index 704a19818e2bb..5dc8086613966 100644
--- a/clients/src/main/java/org/apache/kafka/common/network/SslTransportLayer.java
+++ b/clients/src/main/java/org/apache/kafka/common/network/SslTransportLayer.java
@@ -31,8 +31,10 @@
import javax.net.ssl.SSLEngineResult.Status;
import javax.net.ssl.SSLException;
import javax.net.ssl.SSLHandshakeException;
-import javax.net.ssl.SSLSession;
+import javax.net.ssl.SSLKeyException;
import javax.net.ssl.SSLPeerUnverifiedException;
+import javax.net.ssl.SSLProtocolException;
+import javax.net.ssl.SSLSession;
import org.apache.kafka.common.errors.SslAuthenticationException;
import org.apache.kafka.common.security.auth.KafkaPrincipal;
@@ -45,6 +47,7 @@
*/
public class SslTransportLayer implements TransportLayer {
private enum State {
+ NOT_INITALIZED,
HANDSHAKE,
HANDSHAKE_FAILED,
READY,
@@ -64,12 +67,11 @@ private enum State {
private ByteBuffer netReadBuffer;
private ByteBuffer netWriteBuffer;
private ByteBuffer appReadBuffer;
+ private boolean hasBytesBuffered;
private ByteBuffer emptyBuf = ByteBuffer.allocate(0);
public static SslTransportLayer create(String channelId, SelectionKey key, SSLEngine sslEngine) throws IOException {
- SslTransportLayer transportLayer = new SslTransportLayer(channelId, key, sslEngine);
- transportLayer.startHandshake();
- return transportLayer;
+ return new SslTransportLayer(channelId, key, sslEngine);
}
// Prefer `create`, only use this in tests
@@ -78,6 +80,7 @@ public static SslTransportLayer create(String channelId, SelectionKey key, SSLEn
this.key = key;
this.socketChannel = (SocketChannel) key.channel();
this.sslEngine = sslEngine;
+ this.state = State.NOT_INITALIZED;
final LogContext logContext = new LogContext(String.format("[SslTransportLayer channelId=%s key=%s] ", channelId, key));
this.log = logContext.logger(getClass());
@@ -85,7 +88,7 @@ public static SslTransportLayer create(String channelId, SelectionKey key, SSLEn
// Visible for testing
protected void startHandshake() throws IOException {
- if (state != null)
+ if (state != State.NOT_INITALIZED)
throw new IllegalStateException("startHandshake() can only be called once, state " + state);
this.netReadBuffer = ByteBuffer.allocate(netReadBufferSize());
@@ -153,11 +156,12 @@ public boolean isConnected() {
*/
@Override
public void close() throws IOException {
+ State prevState = state;
if (state == State.CLOSING) return;
state = State.CLOSING;
sslEngine.closeOutbound();
try {
- if (isConnected()) {
+ if (prevState != State.NOT_INITALIZED && isConnected()) {
if (!flush(netWriteBuffer)) {
throw new IOException("Remaining data in the network buffer, can't send SSL close message.");
}
@@ -174,10 +178,13 @@ public void close() throws IOException {
flush(netWriteBuffer);
}
} catch (IOException ie) {
- log.warn("Failed to send SSL Close message", ie);
+ log.debug("Failed to send SSL Close message", ie);
} finally {
socketChannel.socket().close();
socketChannel.close();
+ netReadBuffer = null;
+ netWriteBuffer = null;
+ appReadBuffer = null;
}
}
@@ -239,6 +246,8 @@ protected boolean flush(ByteBuffer buf) throws IOException {
*/
@Override
public void handshake() throws IOException {
+ if (state == State.NOT_INITALIZED)
+ startHandshake();
if (state == State.READY)
throw renegotiationException();
if (state == State.CLOSING)
@@ -254,17 +263,17 @@ public void handshake() throws IOException {
doHandshake();
} catch (SSLException e) {
- handshakeFailure(e, true);
+ maybeProcessHandshakeFailure(e, true, null);
} catch (IOException e) {
maybeThrowSslAuthenticationException();
// this exception could be due to a write. If there is data available to unwrap,
- // process the data so that any SSLExceptions are reported
+ // process the data so that any SSL handshake exceptions are reported
if (handshakeStatus == HandshakeStatus.NEED_UNWRAP && netReadBuffer.position() > 0) {
try {
handshakeUnwrap(false);
} catch (SSLException e1) {
- handshakeFailure(e1, false);
+ maybeProcessHandshakeFailure(e1, false, e);
}
}
// If we get here, this is not a handshake failure, throw the original IOException
@@ -503,13 +512,17 @@ public int read(ByteBuffer dst) throws IOException {
read = readFromAppBuffer(dst);
}
+ boolean readFromNetwork = false;
boolean isClosed = false;
// Each loop reads at most once from the socket.
while (dst.remaining() > 0) {
int netread = 0;
netReadBuffer = Utils.ensureCapacity(netReadBuffer, netReadBufferSize());
- if (netReadBuffer.remaining() > 0)
+ if (netReadBuffer.remaining() > 0) {
netread = readFromSocketChannel();
+ if (netread > 0)
+ readFromNetwork = true;
+ }
while (netReadBuffer.position() > 0) {
netReadBuffer.flip();
@@ -563,6 +576,7 @@ public int read(ByteBuffer dst) throws IOException {
if (netread <= 0 || isClosed)
break;
}
+ updateBytesBuffered(readFromNetwork || read > 0);
// If data has been read and unwrapped, return the data even if end-of-stream, channel will be closed
// on a subsequent poll.
return read;
@@ -793,6 +807,11 @@ protected ByteBuffer netReadBuffer() {
return netReadBuffer;
}
+ // Visibility for testing
+ protected ByteBuffer appReadBuffer() {
+ return appReadBuffer;
+ }
+
/**
* SSL exceptions are propagated as authentication failures so that clients can avoid
* retries and report the failure. If `flush` is true, exceptions are propagated after
@@ -813,6 +832,32 @@ private void handshakeFailure(SSLException sslException, boolean flush) throws I
throw handshakeException;
}
+ // SSL handshake failures are typically thrown as SSLHandshakeException, SSLProtocolException,
+ // SSLPeerUnverifiedException or SSLKeyException if the cause is known. These exceptions indicate
+ // authentication failures (e.g. configuration errors) which should not be retried. But the SSL engine
+ // may also throw exceptions using the base class SSLException in a few cases:
+ // a) If there are no matching ciphers or TLS version or the private key is invalid, client will be
+ // unable to process the server message and an SSLException is thrown:
+ // javax.net.ssl.SSLException: Unrecognized SSL message, plaintext connection?
+ // b) If server closes the connection gracefully during handshake, client may receive close_notify
+ // and and an SSLException is thrown:
+ // javax.net.ssl.SSLException: Received close_notify during handshake
+ // We want to handle a) as a non-retriable SslAuthenticationException and b) as a retriable IOException.
+ // To do this we need to rely on the exception string. Since it is safer to throw a retriable exception
+ // when we are not sure, we will treat only the first exception string as a handshake exception.
+ private void maybeProcessHandshakeFailure(SSLException sslException, boolean flush, IOException ioException) throws IOException {
+ if (sslException instanceof SSLHandshakeException || sslException instanceof SSLProtocolException ||
+ sslException instanceof SSLPeerUnverifiedException || sslException instanceof SSLKeyException ||
+ sslException.getMessage().contains("Unrecognized SSL message"))
+ handshakeFailure(sslException, flush);
+ else if (ioException == null)
+ throw sslException;
+ else {
+ log.debug("SSLException while unwrapping data after IOException, original IOException will be propagated", sslException);
+ throw ioException;
+ }
+ }
+
// If handshake has already failed, throw the authentication exception.
private void maybeThrowSslAuthenticationException() {
if (handshakeException != null)
@@ -826,12 +871,22 @@ public boolean isMute() {
@Override
public boolean hasBytesBuffered() {
- return netReadBuffer.position() != 0 || appReadBuffer.position() != 0;
+ return hasBytesBuffered;
+ }
+
+ // Update `hasBytesBuffered` status. If any bytes were read from the network or
+ // if data was returned from read, `hasBytesBuffered` is set to true if any buffered
+ // data is still remaining. If not, `hasBytesBuffered` is set to false since no progress
+ // can be made until more data is available to read from the network.
+ private void updateBytesBuffered(boolean madeProgress) {
+ if (madeProgress)
+ hasBytesBuffered = netReadBuffer.position() != 0 || appReadBuffer.position() != 0;
+ else
+ hasBytesBuffered = false;
}
@Override
public long transferFrom(FileChannel fileChannel, long position, long count) throws IOException {
return fileChannel.transferTo(position, count, this);
}
-
}
diff --git a/clients/src/main/java/org/apache/kafka/common/network/TransportLayer.java b/clients/src/main/java/org/apache/kafka/common/network/TransportLayer.java
index 3673d21dae6b5..a8a4b8730283c 100644
--- a/clients/src/main/java/org/apache/kafka/common/network/TransportLayer.java
+++ b/clients/src/main/java/org/apache/kafka/common/network/TransportLayer.java
@@ -94,6 +94,7 @@ public interface TransportLayer extends ScatteringByteChannel, GatheringByteChan
/**
* @return true if channel has bytes to be read in any intermediate buffers
+ * which may be processed without reading additional data from the network.
*/
boolean hasBytesBuffered();
diff --git a/clients/src/main/java/org/apache/kafka/common/protocol/CommonFields.java b/clients/src/main/java/org/apache/kafka/common/protocol/CommonFields.java
index 96fa136011d7a..9eddf2b17e668 100644
--- a/clients/src/main/java/org/apache/kafka/common/protocol/CommonFields.java
+++ b/clients/src/main/java/org/apache/kafka/common/protocol/CommonFields.java
@@ -17,7 +17,7 @@
package org.apache.kafka.common.protocol;
import org.apache.kafka.common.protocol.types.Field;
-import org.apache.kafka.common.resource.ResourceNameType;
+import org.apache.kafka.common.resource.PatternType;
public class CommonFields {
public static final Field.Int32 THROTTLE_TIME_MS = new Field.Int32("throttle_time_ms",
@@ -46,8 +46,8 @@ public class CommonFields {
public static final Field.Int8 RESOURCE_TYPE = new Field.Int8("resource_type", "The resource type");
public static final Field.Str RESOURCE_NAME = new Field.Str("resource_name", "The resource name");
public static final Field.NullableStr RESOURCE_NAME_FILTER = new Field.NullableStr("resource_name", "The resource name filter");
- public static final Field.Int8 RESOURCE_NAME_TYPE = new Field.Int8("resource_name_type", "The resource name type", ResourceNameType.LITERAL.code());
- public static final Field.Int8 RESOURCE_NAME_TYPE_FILTER = new Field.Int8("resource_name_type_filter", "The resource name type filter", ResourceNameType.LITERAL.code());
+ public static final Field.Int8 RESOURCE_PATTERN_TYPE = new Field.Int8("resource_pattten_type", "The resource pattern type", PatternType.LITERAL.code());
+ public static final Field.Int8 RESOURCE_PATTERN_TYPE_FILTER = new Field.Int8("resource_pattern_type_filter", "The resource pattern type filter", PatternType.LITERAL.code());
public static final Field.Str PRINCIPAL = new Field.Str("principal", "The ACL principal");
public static final Field.NullableStr PRINCIPAL_FILTER = new Field.NullableStr("principal", "The ACL principal filter");
public static final Field.Str HOST = new Field.Str("host", "The ACL host");
diff --git a/clients/src/main/java/org/apache/kafka/common/protocol/Errors.java b/clients/src/main/java/org/apache/kafka/common/protocol/Errors.java
index 5db1d314be301..9c522dff935ec 100644
--- a/clients/src/main/java/org/apache/kafka/common/protocol/Errors.java
+++ b/clients/src/main/java/org/apache/kafka/common/protocol/Errors.java
@@ -30,6 +30,7 @@
import org.apache.kafka.common.errors.DelegationTokenExpiredException;
import org.apache.kafka.common.errors.DelegationTokenNotFoundException;
import org.apache.kafka.common.errors.DelegationTokenOwnerMismatchException;
+import org.apache.kafka.common.errors.ListenerNotFoundException;
import org.apache.kafka.common.errors.FetchSessionIdNotFoundException;
import org.apache.kafka.common.errors.GroupAuthorizationException;
import org.apache.kafka.common.errors.GroupIdNotFoundException;
@@ -624,7 +625,14 @@ public ApiException build(String message) {
public ApiException build(String message) {
return new InvalidFetchSessionEpochException(message);
}
- });
+ }),
+ LISTENER_NOT_FOUND(72, "There is no listener on the leader broker that matches the listener on which metadata request was processed",
+ new ApiExceptionBuilder() {
+ @Override
+ public ApiException build(String message) {
+ return new ListenerNotFoundException(message);
+ }
+ }),;
private interface ApiExceptionBuilder {
ApiException build(String message);
diff --git a/clients/src/main/java/org/apache/kafka/common/protocol/types/Struct.java b/clients/src/main/java/org/apache/kafka/common/protocol/types/Struct.java
index 7dccc1015aad0..7183aedbd951a 100644
--- a/clients/src/main/java/org/apache/kafka/common/protocol/types/Struct.java
+++ b/clients/src/main/java/org/apache/kafka/common/protocol/types/Struct.java
@@ -111,6 +111,12 @@ public Short getOrElse(Field.Int16 field, short alternative) {
return alternative;
}
+ public Byte getOrElse(Field.Int8 field, byte alternative) {
+ if (hasField(field.name))
+ return getByte(field.name);
+ return alternative;
+ }
+
public Integer getOrElse(Field.Int32 field, int alternative) {
if (hasField(field.name))
return getInt(field.name);
diff --git a/clients/src/main/java/org/apache/kafka/common/record/AbstractLegacyRecordBatch.java b/clients/src/main/java/org/apache/kafka/common/record/AbstractLegacyRecordBatch.java
index 6ac073dd29801..291873e81cc09 100644
--- a/clients/src/main/java/org/apache/kafka/common/record/AbstractLegacyRecordBatch.java
+++ b/clients/src/main/java/org/apache/kafka/common/record/AbstractLegacyRecordBatch.java
@@ -29,7 +29,6 @@
import java.io.IOException;
import java.io.InputStream;
import java.nio.ByteBuffer;
-import java.nio.channels.FileChannel;
import java.util.ArrayDeque;
import java.util.Iterator;
import java.util.NoSuchElementException;
@@ -528,10 +527,10 @@ static class LegacyFileChannelRecordBatch extends FileLogInputStream.FileChannel
LegacyFileChannelRecordBatch(long offset,
byte magic,
- FileChannel channel,
+ FileRecords fileRecords,
int position,
int batchSize) {
- super(offset, magic, channel, position, batchSize);
+ super(offset, magic, fileRecords, position, batchSize);
}
@Override
diff --git a/clients/src/main/java/org/apache/kafka/common/record/BufferSupplier.java b/clients/src/main/java/org/apache/kafka/common/record/BufferSupplier.java
index 2e09f7d1a2cc3..1a6c92c712fba 100644
--- a/clients/src/main/java/org/apache/kafka/common/record/BufferSupplier.java
+++ b/clients/src/main/java/org/apache/kafka/common/record/BufferSupplier.java
@@ -93,4 +93,36 @@ public void close() {
bufferMap.clear();
}
}
+
+ /**
+ * Simple buffer supplier for single-threaded usage. It caches a single buffer, which grows
+ * monotonically as needed to fulfill the allocation request.
+ */
+ public static class GrowableBufferSupplier extends BufferSupplier {
+ private ByteBuffer cachedBuffer;
+
+ @Override
+ public ByteBuffer get(int minCapacity) {
+ if (cachedBuffer != null && cachedBuffer.capacity() >= minCapacity) {
+ ByteBuffer res = cachedBuffer;
+ cachedBuffer = null;
+ return res;
+ } else {
+ cachedBuffer = null;
+ return ByteBuffer.allocate(minCapacity);
+ }
+ }
+
+ @Override
+ public void release(ByteBuffer buffer) {
+ buffer.clear();
+ cachedBuffer = buffer;
+ }
+
+ @Override
+ public void close() {
+ cachedBuffer = null;
+ }
+ }
+
}
diff --git a/clients/src/main/java/org/apache/kafka/common/record/DefaultRecordBatch.java b/clients/src/main/java/org/apache/kafka/common/record/DefaultRecordBatch.java
index 71e668e45da01..19ddb0ef3fa1a 100644
--- a/clients/src/main/java/org/apache/kafka/common/record/DefaultRecordBatch.java
+++ b/clients/src/main/java/org/apache/kafka/common/record/DefaultRecordBatch.java
@@ -28,7 +28,6 @@
import java.io.IOException;
import java.nio.BufferUnderflowException;
import java.nio.ByteBuffer;
-import java.nio.channels.FileChannel;
import java.util.ArrayList;
import java.util.Collections;
import java.util.Iterator;
@@ -523,12 +522,18 @@ static int estimateBatchSizeUpperBound(ByteBuffer key, ByteBuffer value, Header[
return RECORD_BATCH_OVERHEAD + DefaultRecord.recordSizeUpperBound(key, value, headers);
}
- static int incrementSequence(int baseSequence, int increment) {
+ public static int incrementSequence(int baseSequence, int increment) {
if (baseSequence > Integer.MAX_VALUE - increment)
return increment - (Integer.MAX_VALUE - baseSequence) - 1;
return baseSequence + increment;
}
+ public static int decrementSequence(int baseSequence, int decrement) {
+ if (baseSequence < decrement)
+ return Integer.MAX_VALUE - (decrement - baseSequence) + 1;
+ return baseSequence - decrement;
+ }
+
private abstract class RecordIterator implements CloseableIterator {
private final Long logAppendTime;
private final long baseOffset;
@@ -586,10 +591,10 @@ static class DefaultFileChannelRecordBatch extends FileLogInputStream.FileChanne
DefaultFileChannelRecordBatch(long offset,
byte magic,
- FileChannel channel,
+ FileRecords fileRecords,
int position,
int batchSize) {
- super(offset, magic, channel, position, batchSize);
+ super(offset, magic, fileRecords, position, batchSize);
}
@Override
diff --git a/clients/src/main/java/org/apache/kafka/common/record/FileLogInputStream.java b/clients/src/main/java/org/apache/kafka/common/record/FileLogInputStream.java
index 92e8864a183b6..472c7a7ac3e13 100644
--- a/clients/src/main/java/org/apache/kafka/common/record/FileLogInputStream.java
+++ b/clients/src/main/java/org/apache/kafka/common/record/FileLogInputStream.java
@@ -82,9 +82,9 @@ public FileChannelRecordBatch nextBatch() throws IOException {
final FileChannelRecordBatch batch;
if (magic < RecordBatch.MAGIC_VALUE_V2)
- batch = new LegacyFileChannelRecordBatch(offset, magic, channel, position, size);
+ batch = new LegacyFileChannelRecordBatch(offset, magic, fileRecords, position, size);
else
- batch = new DefaultFileChannelRecordBatch(offset, magic, channel, position, size);
+ batch = new DefaultFileChannelRecordBatch(offset, magic, fileRecords, position, size);
position += batch.sizeInBytes();
return batch;
@@ -98,7 +98,7 @@ public FileChannelRecordBatch nextBatch() throws IOException {
public abstract static class FileChannelRecordBatch extends AbstractRecordBatch {
protected final long offset;
protected final byte magic;
- protected final FileChannel channel;
+ protected final FileRecords fileRecords;
protected final int position;
protected final int batchSize;
@@ -107,12 +107,12 @@ public abstract static class FileChannelRecordBatch extends AbstractRecordBatch
FileChannelRecordBatch(long offset,
byte magic,
- FileChannel channel,
+ FileRecords fileRecords,
int position,
int batchSize) {
this.offset = offset;
this.magic = magic;
- this.channel = channel;
+ this.fileRecords = fileRecords;
this.position = position;
this.batchSize = batchSize;
}
@@ -173,14 +173,14 @@ public int sizeInBytes() {
@Override
public void writeTo(ByteBuffer buffer) {
+ FileChannel channel = fileRecords.channel();
try {
int limit = buffer.limit();
buffer.limit(buffer.position() + sizeInBytes());
Utils.readFully(channel, buffer, position);
buffer.limit(limit);
} catch (IOException e) {
- throw new KafkaException("Failed to read record batch at position " + position + " from file channel " +
- channel, e);
+ throw new KafkaException("Failed to read record batch at position " + position + " from " + fileRecords, e);
}
}
@@ -207,13 +207,14 @@ protected RecordBatch loadBatchHeader() {
}
private RecordBatch loadBatchWithSize(int size, String description) {
+ FileChannel channel = fileRecords.channel();
try {
ByteBuffer buffer = ByteBuffer.allocate(size);
Utils.readFullyOrFail(channel, buffer, position, description);
buffer.rewind();
return toMemoryRecordBatch(buffer);
} catch (IOException e) {
- throw new KafkaException(e);
+ throw new KafkaException("Failed to load record batch at position " + position + " from " + fileRecords, e);
}
}
@@ -226,14 +227,19 @@ public boolean equals(Object o) {
FileChannelRecordBatch that = (FileChannelRecordBatch) o;
+ FileChannel channel = fileRecords == null ? null : fileRecords.channel();
+ FileChannel thatChannel = that.fileRecords == null ? null : that.fileRecords.channel();
+
return offset == that.offset &&
position == that.position &&
batchSize == that.batchSize &&
- (channel == null ? that.channel == null : channel.equals(that.channel));
+ (channel == null ? thatChannel == null : channel.equals(thatChannel));
}
@Override
public int hashCode() {
+ FileChannel channel = fileRecords == null ? null : fileRecords.channel();
+
int result = (int) (offset ^ (offset >>> 32));
result = 31 * result + (channel != null ? channel.hashCode() : 0);
result = 31 * result + position;
diff --git a/clients/src/main/java/org/apache/kafka/common/record/FileRecords.java b/clients/src/main/java/org/apache/kafka/common/record/FileRecords.java
index e44d5d93be814..df38ac78ba37e 100644
--- a/clients/src/main/java/org/apache/kafka/common/record/FileRecords.java
+++ b/clients/src/main/java/org/apache/kafka/common/record/FileRecords.java
@@ -109,14 +109,12 @@ public FileChannel channel() {
*
* @param buffer The buffer to write the batches to
* @param position Position in the buffer to read from
- * @return The same buffer
* @throws IOException If an I/O error occurs, see {@link FileChannel#read(ByteBuffer, long)} for details on the
* possible exceptions
*/
- public ByteBuffer readInto(ByteBuffer buffer, int position) throws IOException {
+ public void readInto(ByteBuffer buffer, int position) throws IOException {
Utils.readFully(channel, buffer, position + this.start);
buffer.flip();
- return buffer;
}
/**
@@ -131,7 +129,7 @@ public ByteBuffer readInto(ByteBuffer buffer, int position) throws IOException {
* @param size The number of bytes after the start position to include
* @return A sliced wrapper on this message set limited based on the given position and size
*/
- public FileRecords read(int position, int size) throws IOException {
+ public FileRecords slice(int position, int size) throws IOException {
if (position < 0)
throw new IllegalArgumentException("Invalid position: " + position + " in read from " + file);
if (size < 0)
@@ -356,7 +354,14 @@ public String toString() {
")";
}
- private Iterable batchesFrom(final int start) {
+ /**
+ * Get an iterator over the record batches in the file, starting at a specific position. This is similar to
+ * {@link #batches()} except that callers specify a particular position to start reading the batches from. This
+ * method must be used with caution: the start position passed in must be a known start of a batch.
+ * @param start The position to start record iteration from; must be a known position for start of a batch
+ * @return An iterator over batches starting from {@code start}
+ */
+ public Iterable batchesFrom(final int start) {
return new Iterable() {
@Override
public Iterator iterator() {
diff --git a/clients/src/main/java/org/apache/kafka/common/record/LazyDownConversionRecords.java b/clients/src/main/java/org/apache/kafka/common/record/LazyDownConversionRecords.java
index da14b5b494f2d..d58689de119a8 100644
--- a/clients/src/main/java/org/apache/kafka/common/record/LazyDownConversionRecords.java
+++ b/clients/src/main/java/org/apache/kafka/common/record/LazyDownConversionRecords.java
@@ -21,7 +21,6 @@
import org.apache.kafka.common.utils.Time;
import java.util.ArrayList;
-import java.util.Collections;
import java.util.List;
import java.util.Objects;
@@ -57,13 +56,15 @@ public LazyDownConversionRecords(TopicPartition topicPartition, Records records,
// need to make sure that we are able to accommodate one full batch of down-converted messages. The way we achieve
// this is by having sizeInBytes method factor in the size of the first down-converted batch and return at least
// its size.
- AbstractIterator extends RecordBatch> it = records.batchIterator();
+ java.util.Iterator it = iterator(0);
if (it.hasNext()) {
- firstConvertedBatch = RecordsUtil.downConvert(Collections.singletonList(it.peek()), toMagic, firstOffset, time);
+ firstConvertedBatch = it.next();
sizeInBytes = Math.max(records.sizeInBytes(), firstConvertedBatch.records().sizeInBytes());
} else {
+ // If there are no messages we got after down-conversion, make sure we are able to send at least an overflow
+ // message to the consumer. Typically, the consumer would need to increase the fetch size in such cases.
firstConvertedBatch = null;
- sizeInBytes = 0;
+ sizeInBytes = LazyDownConversionRecordsSend.MIN_OVERFLOW_MESSAGE_LENGTH;
}
}
@@ -148,21 +149,28 @@ protected ConvertedRecords makeNext() {
return convertedBatch;
}
- if (!batchIterator.hasNext())
- return allDone();
-
- // Figure out batches we should down-convert based on the size constraints
- List batches = new ArrayList<>();
- boolean isFirstBatch = true;
- long sizeSoFar = 0;
- while (batchIterator.hasNext() &&
- (isFirstBatch || (batchIterator.peek().sizeInBytes() + sizeSoFar) <= maximumReadSize)) {
- RecordBatch currentBatch = batchIterator.next();
- batches.add(currentBatch);
- sizeSoFar += currentBatch.sizeInBytes();
- isFirstBatch = false;
+ while (batchIterator.hasNext()) {
+ List batches = new ArrayList<>();
+ boolean isFirstBatch = true;
+ long sizeSoFar = 0;
+
+ // Figure out batches we should down-convert based on the size constraints
+ while (batchIterator.hasNext() &&
+ (isFirstBatch || (batchIterator.peek().sizeInBytes() + sizeSoFar) <= maximumReadSize)) {
+ RecordBatch currentBatch = batchIterator.next();
+ batches.add(currentBatch);
+ sizeSoFar += currentBatch.sizeInBytes();
+ isFirstBatch = false;
+ }
+ ConvertedRecords convertedRecords = RecordsUtil.downConvert(batches, toMagic, firstOffset, time);
+ // During conversion, it is possible that we drop certain batches because they do not have an equivalent
+ // representation in the message format we want to convert to. For example, V0 and V1 message formats
+ // have no notion of transaction markers which were introduced in V2 so they get dropped during conversion.
+ // We return converted records only when we have at least one valid batch of messages after conversion.
+ if (convertedRecords.records().sizeInBytes() > 0)
+ return convertedRecords;
}
- return RecordsUtil.downConvert(batches, toMagic, firstOffset, time);
+ return allDone();
}
}
}
diff --git a/clients/src/main/java/org/apache/kafka/common/record/LazyDownConversionRecordsSend.java b/clients/src/main/java/org/apache/kafka/common/record/LazyDownConversionRecordsSend.java
index b78211418acfe..f0fab7d876d92 100644
--- a/clients/src/main/java/org/apache/kafka/common/record/LazyDownConversionRecordsSend.java
+++ b/clients/src/main/java/org/apache/kafka/common/record/LazyDownConversionRecordsSend.java
@@ -20,7 +20,6 @@
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
-import java.io.EOFException;
import java.io.IOException;
import java.nio.ByteBuffer;
import java.nio.channels.GatheringByteChannel;
@@ -33,6 +32,7 @@
public final class LazyDownConversionRecordsSend extends RecordsSend {
private static final Logger log = LoggerFactory.getLogger(LazyDownConversionRecordsSend.class);
private static final int MAX_READ_SIZE = 128 * 1024;
+ static final int MIN_OVERFLOW_MESSAGE_LENGTH = Records.LOG_OVERHEAD;
private RecordConversionStats recordConversionStats;
private RecordsSend convertedRecordsWriter;
@@ -49,41 +49,31 @@ public LazyDownConversionRecordsSend(String destination, LazyDownConversionRecor
public long writeTo(GatheringByteChannel channel, long previouslyWritten, int remaining) throws IOException {
if (convertedRecordsWriter == null || convertedRecordsWriter.completed()) {
MemoryRecords convertedRecords;
-
// Check if we have more chunks left to down-convert
if (convertedRecordsIterator.hasNext()) {
// Get next chunk of down-converted messages
ConvertedRecords recordsAndStats = convertedRecordsIterator.next();
convertedRecords = recordsAndStats.records();
-
- int sizeOfFirstConvertedBatch = convertedRecords.batchIterator().next().sizeInBytes();
- if (previouslyWritten == 0 && sizeOfFirstConvertedBatch > size())
- throw new EOFException("Unable to send first batch completely." +
- " maximum_size: " + size() +
- " converted_records_size: " + sizeOfFirstConvertedBatch);
-
recordConversionStats.add(recordsAndStats.recordConversionStats());
- log.debug("Got lazy converted records for {" + topicPartition() + "} with length=" + convertedRecords.sizeInBytes());
+ log.debug("Down-converted records for partition {} with length={}", topicPartition(), convertedRecords.sizeInBytes());
} else {
- if (previouslyWritten == 0)
- throw new EOFException("Unable to get the first batch of down-converted records");
-
- // We do not have any records left to down-convert. Construct a "fake" message for the length remaining.
+ // We do not have any records left to down-convert. Construct an overflow message for the length remaining.
// This message will be ignored by the consumer because its length will be past the length of maximum
// possible response size.
// DefaultRecordBatch =>
// BaseOffset => Int64
// Length => Int32
// ...
- // TODO: check if there is a better way to encapsulate this logic, perhaps in DefaultRecordBatch
- log.debug("Constructing fake message batch for topic-partition {" + topicPartition() + "} for remaining length " + remaining);
- int minLength = (Long.SIZE / Byte.SIZE) + (Integer.SIZE / Byte.SIZE);
- ByteBuffer fakeMessageBatch = ByteBuffer.allocate(Math.max(minLength, Math.min(remaining + 1, MAX_READ_SIZE)));
- fakeMessageBatch.putLong(-1L);
- fakeMessageBatch.putInt(remaining + 1);
- convertedRecords = MemoryRecords.readableRecords(fakeMessageBatch);
- }
+ ByteBuffer overflowMessageBatch = ByteBuffer.allocate(
+ Math.max(MIN_OVERFLOW_MESSAGE_LENGTH, Math.min(remaining + 1, MAX_READ_SIZE)));
+ overflowMessageBatch.putLong(-1L);
+ // Fill in the length of the overflow batch. A valid batch must be at least as long as the minimum batch
+ // overhead.
+ overflowMessageBatch.putInt(Math.max(remaining + 1, DefaultRecordBatch.RECORD_BATCH_OVERHEAD));
+ convertedRecords = MemoryRecords.readableRecords(overflowMessageBatch);
+ log.debug("Constructed overflow message batch for partition {} with length={}", topicPartition(), remaining);
+ }
convertedRecordsWriter = new DefaultRecordsSend(destination(), convertedRecords, Math.min(convertedRecords.sizeInBytes(), remaining));
}
return convertedRecordsWriter.writeTo(channel);
diff --git a/clients/src/main/java/org/apache/kafka/common/record/MemoryRecords.java b/clients/src/main/java/org/apache/kafka/common/record/MemoryRecords.java
index 55a471149c6ca..af62e09c57ecd 100644
--- a/clients/src/main/java/org/apache/kafka/common/record/MemoryRecords.java
+++ b/clients/src/main/java/org/apache/kafka/common/record/MemoryRecords.java
@@ -160,20 +160,14 @@ public FilterResult filterTo(TopicPartition partition, RecordFilter filter, Byte
private static FilterResult filterTo(TopicPartition partition, Iterable batches,
RecordFilter filter, ByteBuffer destinationBuffer, int maxRecordBatchSize,
BufferSupplier decompressionBufferSupplier) {
- long maxTimestamp = RecordBatch.NO_TIMESTAMP;
- long maxOffset = -1L;
- long shallowOffsetOfMaxTimestamp = -1L;
- int messagesRead = 0;
- int bytesRead = 0; // bytes processed from `batches`
- int messagesRetained = 0;
- int bytesRetained = 0;
-
+ FilterResult filterResult = new FilterResult(destinationBuffer);
ByteBufferOutputStream bufferOutputStream = new ByteBufferOutputStream(destinationBuffer);
for (MutableRecordBatch batch : batches) {
- bytesRead += batch.sizeInBytes();
-
+ long maxOffset = -1L;
BatchRetention batchRetention = filter.checkBatchRetention(batch);
+ filterResult.bytesRead += batch.sizeInBytes();
+
if (batchRetention == BatchRetention.DELETE)
continue;
@@ -189,7 +183,7 @@ private static FilterResult filterTo(TopicPartition partition, Iterable iterator = batch.streamingIterator(decompressionBufferSupplier)) {
while (iterator.hasNext()) {
Record record = iterator.next();
- messagesRead += 1;
+ filterResult.messagesRead += 1;
if (filter.shouldRetainRecord(batch, record)) {
// Check for log corruption due to KAFKA-4298. If we find it, make sure that we overwrite
@@ -210,20 +204,11 @@ private static FilterResult filterTo(TopicPartition partition, Iterable maxTimestamp) {
- maxTimestamp = batch.maxTimestamp();
- shallowOffsetOfMaxTimestamp = batch.lastOffset();
- }
+ filterResult.updateRetainedBatchMetadata(batch, retainedRecords.size(), false);
} else {
MemoryRecordsBuilder builder = buildRetainedRecordsInto(batch, retainedRecords, bufferOutputStream);
MemoryRecords records = builder.build();
int filteredBatchSize = records.sizeInBytes();
-
- messagesRetained += retainedRecords.size();
- bytesRetained += filteredBatchSize;
-
if (filteredBatchSize > batch.sizeInBytes() && filteredBatchSize > maxRecordBatchSize)
log.warn("Record batch from {} with last offset {} exceeded max record batch size {} after cleaning " +
"(new size is {}). Consumers with version earlier than 0.10.1.0 may need to " +
@@ -231,10 +216,8 @@ private static FilterResult filterTo(TopicPartition partition, Iterable maxTimestamp) {
- maxTimestamp = info.maxTimestamp;
- shallowOffsetOfMaxTimestamp = info.shallowOffsetOfMaxTimestamp;
- }
+ filterResult.updateRetainedBatchMetadata(info.maxTimestamp, info.shallowOffsetOfMaxTimestamp,
+ maxOffset, retainedRecords.size(), filteredBatchSize);
}
} else if (batchRetention == BatchRetention.RETAIN_EMPTY) {
if (batchMagic < RecordBatch.MAGIC_VALUE_V2)
@@ -245,18 +228,19 @@ private static FilterResult filterTo(TopicPartition partition, Iterable this.maxTimestamp) {
+ this.maxTimestamp = maxTimestamp;
+ this.shallowOffsetOfMaxTimestamp = shallowOffsetOfMaxTimestamp;
+ }
+ this.maxOffset = Math.max(maxOffset, this.maxOffset);
+ this.messagesRetained += messagesRetained;
+ this.bytesRetained += bytesRetained;
+ }
+
+ private void validateBatchMetadata(long maxTimestamp, long shallowOffsetOfMaxTimestamp, long maxOffset) {
+ if (maxTimestamp != RecordBatch.NO_TIMESTAMP && shallowOffsetOfMaxTimestamp < 0)
+ throw new IllegalArgumentException("shallowOffset undefined for maximum timestamp " + maxTimestamp);
+ if (maxOffset < 0)
+ throw new IllegalArgumentException("maxOffset undefined");
+ }
+
+ public ByteBuffer outputBuffer() {
+ return outputBuffer;
+ }
+
+ public int messagesRead() {
+ return messagesRead;
+ }
+
+ public int bytesRead() {
+ return bytesRead;
+ }
+
+ public int messagesRetained() {
+ return messagesRetained;
+ }
+
+ public int bytesRetained() {
+ return bytesRetained;
+ }
+
+ public long maxOffset() {
+ return maxOffset;
+ }
+
+ public long maxTimestamp() {
+ return maxTimestamp;
+ }
+
+ public long shallowOffsetOfMaxTimestamp() {
+ return shallowOffsetOfMaxTimestamp;
}
}
diff --git a/clients/src/main/java/org/apache/kafka/common/record/RecordVersion.java b/clients/src/main/java/org/apache/kafka/common/record/RecordVersion.java
index 1f80d623ca9fd..8406d5331c8a8 100644
--- a/clients/src/main/java/org/apache/kafka/common/record/RecordVersion.java
+++ b/clients/src/main/java/org/apache/kafka/common/record/RecordVersion.java
@@ -34,6 +34,15 @@ public enum RecordVersion {
this.value = (byte) value;
}
+ /**
+ * Check whether this version precedes another version.
+ *
+ * @return true only if the magic value is less than the other's
+ */
+ public boolean precedes(RecordVersion other) {
+ return this.value < other.value;
+ }
+
public static RecordVersion lookup(byte value) {
if (value < 0 || value >= VALUES.length)
throw new IllegalArgumentException("Unknown record version: " + value);
diff --git a/clients/src/main/java/org/apache/kafka/common/record/RecordsUtil.java b/clients/src/main/java/org/apache/kafka/common/record/RecordsUtil.java
index c9b739413175c..291f241152188 100644
--- a/clients/src/main/java/org/apache/kafka/common/record/RecordsUtil.java
+++ b/clients/src/main/java/org/apache/kafka/common/record/RecordsUtil.java
@@ -17,6 +17,7 @@
package org.apache.kafka.common.record;
import org.apache.kafka.common.utils.Time;
+import org.apache.kafka.common.utils.Utils;
import java.nio.ByteBuffer;
import java.util.ArrayList;
@@ -73,9 +74,11 @@ protected static ConvertedRecords downConvert(Iterable extends
ByteBuffer buffer = ByteBuffer.allocate(totalSizeEstimate);
long temporaryMemoryBytes = 0;
int numRecordsConverted = 0;
+
for (RecordBatchAndRecords recordBatchAndRecords : recordBatchAndRecordsList) {
temporaryMemoryBytes += recordBatchAndRecords.batch.sizeInBytes();
if (recordBatchAndRecords.batch.magic() <= toMagic) {
+ buffer = Utils.ensureCapacity(buffer, buffer.position() + recordBatchAndRecords.batch.sizeInBytes());
recordBatchAndRecords.batch.writeTo(buffer);
} else {
MemoryRecordsBuilder builder = convertRecordBatch(toMagic, buffer, recordBatchAndRecords);
diff --git a/clients/src/main/java/org/apache/kafka/common/requests/AlterConfigsRequest.java b/clients/src/main/java/org/apache/kafka/common/requests/AlterConfigsRequest.java
index ed93b150bde43..ff1d0625c4dba 100644
--- a/clients/src/main/java/org/apache/kafka/common/requests/AlterConfigsRequest.java
+++ b/clients/src/main/java/org/apache/kafka/common/requests/AlterConfigsRequest.java
@@ -17,6 +17,7 @@
package org.apache.kafka.common.requests;
+import org.apache.kafka.common.config.ConfigResource;
import org.apache.kafka.common.protocol.ApiKeys;
import org.apache.kafka.common.protocol.types.ArrayOf;
import org.apache.kafka.common.protocol.types.Field;
@@ -29,6 +30,7 @@
import java.util.HashMap;
import java.util.List;
import java.util.Map;
+import java.util.Objects;
import static org.apache.kafka.common.protocol.types.Type.BOOLEAN;
import static org.apache.kafka.common.protocol.types.Type.INT8;
@@ -73,7 +75,7 @@ public static class Config {
private final Collection entries;
public Config(Collection entries) {
- this.entries = entries;
+ this.entries = Objects.requireNonNull(entries, "entries");
}
public Collection entries() {
@@ -86,8 +88,8 @@ public static class ConfigEntry {
private final String value;
public ConfigEntry(String name, String value) {
- this.name = name;
- this.value = value;
+ this.name = Objects.requireNonNull(name, "name");
+ this.value = Objects.requireNonNull(value, "value");
}
public String name() {
@@ -102,12 +104,12 @@ public String value() {
public static class Builder extends AbstractRequest.Builder {
- private final Map configs;
+ private final Map configs;
private final boolean validateOnly;
- public Builder(Map configs, boolean validateOnly) {
+ public Builder(Map configs, boolean validateOnly) {
super(ApiKeys.ALTER_CONFIGS);
- this.configs = configs;
+ this.configs = Objects.requireNonNull(configs, "configs");
this.validateOnly = validateOnly;
}
@@ -117,12 +119,12 @@ public AlterConfigsRequest build(short version) {
}
}
- private final Map configs;
+ private final Map configs;
private final boolean validateOnly;
- public AlterConfigsRequest(short version, Map configs, boolean validateOnly) {
+ public AlterConfigsRequest(short version, Map configs, boolean validateOnly) {
super(version);
- this.configs = configs;
+ this.configs = Objects.requireNonNull(configs, "configs");
this.validateOnly = validateOnly;
}
@@ -134,9 +136,9 @@ public AlterConfigsRequest(Struct struct, short version) {
for (Object resourcesObj : resourcesArray) {
Struct resourcesStruct = (Struct) resourcesObj;
- ResourceType resourceType = ResourceType.forId(resourcesStruct.getByte(RESOURCE_TYPE_KEY_NAME));
+ ConfigResource.Type resourceType = ConfigResource.Type.forId(resourcesStruct.getByte(RESOURCE_TYPE_KEY_NAME));
String resourceName = resourcesStruct.getString(RESOURCE_NAME_KEY_NAME);
- Resource resource = new Resource(resourceType, resourceName);
+ ConfigResource resource = new ConfigResource(resourceType, resourceName);
Object[] configEntriesArray = resourcesStruct.getArray(CONFIG_ENTRIES_KEY_NAME);
List configEntries = new ArrayList<>(configEntriesArray.length);
@@ -151,7 +153,7 @@ public AlterConfigsRequest(Struct struct, short version) {
}
}
- public Map configs() {
+ public Map configs() {
return configs;
}
@@ -164,10 +166,10 @@ protected Struct toStruct() {
Struct struct = new Struct(ApiKeys.ALTER_CONFIGS.requestSchema(version()));
struct.set(VALIDATE_ONLY_KEY_NAME, validateOnly);
List resourceStructs = new ArrayList<>(configs.size());
- for (Map.Entry entry : configs.entrySet()) {
+ for (Map.Entry