Merged in trunk

This commit is contained in:
Geoff Anderson 2015-06-16 14:31:10 -07:00
commit 69dd7be5e8
230 changed files with 5413 additions and 2775 deletions

View File

@ -93,6 +93,9 @@ Please note for this to work you should create/update `~/.gradle/gradle.properti
signing.password= signing.password=
signing.secretKeyRingFile= signing.secretKeyRingFile=
### Publishing the jars without signing to a local repository ###
./gradlew -Dorg.gradle.project.skipSigning=true -Dorg.gradle.project.mavenUrl=file://path/to/repo uploadArchivesAll
### Building the test jar ### ### Building the test jar ###
./gradlew testJar ./gradlew testJar

View File

@ -30,11 +30,14 @@ allprojects {
} }
ext { ext {
gradleVersion = "2.0" gradleVersion = "2.4"
}
def isVerificationRequired(project) { skipSigning = project.hasProperty('skipSigning') && skipSigning.toBoolean()
project.gradle.startParameter.taskNames.any { it.contains("upload") } shouldSign = !skipSigning && !version.endsWith("SNAPSHOT")
mavenUrl = project.hasProperty('mavenUrl') ? project.mavenUrl : ''
mavenUsername = project.hasProperty('mavenUsername') ? project.mavenUsername : ''
mavenPassword = project.hasProperty('mavenPassword') ? project.mavenPassword : ''
} }
apply from: file('wrapper.gradle') apply from: file('wrapper.gradle')
@ -47,32 +50,33 @@ subprojects {
apply plugin: 'maven' apply plugin: 'maven'
apply plugin: 'signing' apply plugin: 'signing'
sourceCompatibility = 1.6
licenseTest.onlyIf { isVerificationRequired(project) } licenseTest.onlyIf { isVerificationRequired(project) }
uploadArchives { uploadArchives {
repositories { repositories {
signing { signing {
if (isVerificationRequired(project)) { required { shouldSign }
sign configurations.archives sign configurations.archives
// To test locally, replace mavenUrl in ~/.gradle/gradle.properties to file://localhost/tmp/myRepo/ // To test locally, replace mavenUrl in ~/.gradle/gradle.properties to file://localhost/tmp/myRepo/
mavenDeployer { mavenDeployer {
beforeDeployment { MavenDeployment deployment -> signing.signPom(deployment) } beforeDeployment { MavenDeployment deployment -> signing.signPom(deployment) }
repository(url: "${mavenUrl}") { repository(url: "${mavenUrl}") {
authentication(userName: "${mavenUsername}", password: "${mavenPassword}") authentication(userName: "${mavenUsername}", password: "${mavenPassword}")
} }
afterEvaluate { afterEvaluate {
pom.artifactId = "${archivesBaseName}" pom.artifactId = "${archivesBaseName}"
pom.project { pom.project {
name 'Apache Kafka' name 'Apache Kafka'
packaging 'jar' packaging 'jar'
url 'http://kafka.apache.org' url 'http://kafka.apache.org'
licenses { licenses {
license { license {
name 'The Apache Software License, Version 2.0' name 'The Apache Software License, Version 2.0'
url 'http://www.apache.org/licenses/LICENSE-2.0.txt' url 'http://www.apache.org/licenses/LICENSE-2.0.txt'
distribution 'repo' distribution 'repo'
}
} }
} }
} }
@ -205,7 +209,7 @@ project(':core') {
compile project(':clients') compile project(':clients')
compile "org.scala-lang:scala-library:$scalaVersion" compile "org.scala-lang:scala-library:$scalaVersion"
compile 'org.apache.zookeeper:zookeeper:3.4.6' compile 'org.apache.zookeeper:zookeeper:3.4.6'
compile 'com.101tec:zkclient:0.3' compile 'com.101tec:zkclient:0.5'
compile 'com.yammer.metrics:metrics-core:2.2.0' compile 'com.yammer.metrics:metrics-core:2.2.0'
compile 'net.sf.jopt-simple:jopt-simple:3.2' compile 'net.sf.jopt-simple:jopt-simple:3.2'
@ -224,7 +228,7 @@ project(':core') {
} }
testRuntime "$slf4jlog4j" testRuntime "$slf4jlog4j"
zinc 'com.typesafe.zinc:zinc:0.3.1' zinc 'com.typesafe.zinc:zinc:0.3.7'
} }
configurations { configurations {
@ -350,7 +354,7 @@ project(':clients') {
dependencies { dependencies {
compile "org.slf4j:slf4j-api:1.7.6" compile "org.slf4j:slf4j-api:1.7.6"
compile 'org.xerial.snappy:snappy-java:1.1.1.6' compile 'org.xerial.snappy:snappy-java:1.1.1.7'
compile 'net.jpountz.lz4:lz4:1.2.0' compile 'net.jpountz.lz4:lz4:1.2.0'
compile 'net.sourceforge.argparse4j:argparse4j:0.5.0' compile 'net.sourceforge.argparse4j:argparse4j:0.5.0'
compile 'com.googlecode.json-simple:json-simple:1.1.1' compile 'com.googlecode.json-simple:json-simple:1.1.1'

View File

@ -12,16 +12,21 @@
*/ */
package org.apache.kafka.clients; package org.apache.kafka.clients;
import java.io.Closeable;
import java.net.InetSocketAddress; import java.net.InetSocketAddress;
import java.util.ArrayList; import java.util.ArrayList;
import java.util.List; import java.util.List;
import java.util.concurrent.atomic.AtomicReference;
import org.apache.kafka.common.config.ConfigException; import org.apache.kafka.common.config.ConfigException;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
import static org.apache.kafka.common.utils.Utils.getHost; import static org.apache.kafka.common.utils.Utils.getHost;
import static org.apache.kafka.common.utils.Utils.getPort; import static org.apache.kafka.common.utils.Utils.getPort;
public class ClientUtils { public class ClientUtils {
private static final Logger log = LoggerFactory.getLogger(ClientUtils.class);
public static List<InetSocketAddress> parseAndValidateAddresses(List<String> urls) { public static List<InetSocketAddress> parseAndValidateAddresses(List<String> urls) {
List<InetSocketAddress> addresses = new ArrayList<InetSocketAddress>(); List<InetSocketAddress> addresses = new ArrayList<InetSocketAddress>();
@ -45,4 +50,15 @@ public class ClientUtils {
throw new ConfigException("No bootstrap urls given in " + CommonClientConfigs.BOOTSTRAP_SERVERS_CONFIG); throw new ConfigException("No bootstrap urls given in " + CommonClientConfigs.BOOTSTRAP_SERVERS_CONFIG);
return addresses; return addresses;
} }
public static void closeQuietly(Closeable c, String name, AtomicReference<Throwable> firstException) {
if (c != null) {
try {
c.close();
} catch (Throwable t) {
firstException.compareAndSet(null, t);
log.error("Failed to close " + name, t);
}
}
}
} }

View File

@ -21,22 +21,22 @@ import java.util.Map;
*/ */
final class ClusterConnectionStates { final class ClusterConnectionStates {
private final long reconnectBackoffMs; private final long reconnectBackoffMs;
private final Map<Integer, NodeConnectionState> nodeState; private final Map<String, NodeConnectionState> nodeState;
public ClusterConnectionStates(long reconnectBackoffMs) { public ClusterConnectionStates(long reconnectBackoffMs) {
this.reconnectBackoffMs = reconnectBackoffMs; this.reconnectBackoffMs = reconnectBackoffMs;
this.nodeState = new HashMap<Integer, NodeConnectionState>(); this.nodeState = new HashMap<String, NodeConnectionState>();
} }
/** /**
* Return true iff we can currently initiate a new connection to the given node. This will be the case if we are not * Return true iff we can currently initiate a new connection. This will be the case if we are not
* connected and haven't been connected for at least the minimum reconnection backoff period. * connected and haven't been connected for at least the minimum reconnection backoff period.
* @param node The node id to check * @param id The connection id to check
* @param now The current time in MS * @param now The current time in MS
* @return true if we can initiate a new connection * @return true if we can initiate a new connection
*/ */
public boolean canConnect(int node, long now) { public boolean canConnect(String id, long now) {
NodeConnectionState state = nodeState.get(node); NodeConnectionState state = nodeState.get(id);
if (state == null) if (state == null)
return true; return true;
else else
@ -45,11 +45,11 @@ final class ClusterConnectionStates {
/** /**
* Return true if we are disconnected from the given node and can't re-establish a connection yet * Return true if we are disconnected from the given node and can't re-establish a connection yet
* @param node The node to check * @param id The connection to check
* @param now The current time in ms * @param now The current time in ms
*/ */
public boolean isBlackedOut(int node, long now) { public boolean isBlackedOut(String id, long now) {
NodeConnectionState state = nodeState.get(node); NodeConnectionState state = nodeState.get(id);
if (state == null) if (state == null)
return false; return false;
else else
@ -60,11 +60,11 @@ final class ClusterConnectionStates {
* Returns the number of milliseconds to wait, based on the connection state, before attempting to send data. When * Returns the number of milliseconds to wait, based on the connection state, before attempting to send data. When
* disconnected, this respects the reconnect backoff time. When connecting or connected, this handles slow/stalled * disconnected, this respects the reconnect backoff time. When connecting or connected, this handles slow/stalled
* connections. * connections.
* @param node The node to check * @param id The connection to check
* @param now The current time in ms * @param now The current time in ms
*/ */
public long connectionDelay(int node, long now) { public long connectionDelay(String id, long now) {
NodeConnectionState state = nodeState.get(node); NodeConnectionState state = nodeState.get(id);
if (state == null) return 0; if (state == null) return 0;
long timeWaited = now - state.lastConnectAttemptMs; long timeWaited = now - state.lastConnectAttemptMs;
if (state.state == ConnectionState.DISCONNECTED) { if (state.state == ConnectionState.DISCONNECTED) {
@ -77,67 +77,67 @@ final class ClusterConnectionStates {
} }
/** /**
* Enter the connecting state for the given node. * Enter the connecting state for the given connection.
* @param node The id of the node we are connecting to * @param id The id of the connection
* @param now The current time. * @param now The current time.
*/ */
public void connecting(int node, long now) { public void connecting(String id, long now) {
nodeState.put(node, new NodeConnectionState(ConnectionState.CONNECTING, now)); nodeState.put(id, new NodeConnectionState(ConnectionState.CONNECTING, now));
} }
/** /**
* Return true iff we have a connection to the give node * Return true iff a specific connection is connected
* @param node The id of the node to check * @param id The id of the connection to check
*/ */
public boolean isConnected(int node) { public boolean isConnected(String id) {
NodeConnectionState state = nodeState.get(node); NodeConnectionState state = nodeState.get(id);
return state != null && state.state == ConnectionState.CONNECTED; return state != null && state.state == ConnectionState.CONNECTED;
} }
/** /**
* Return true iff we are in the process of connecting to the given node * Return true iff we are in the process of connecting
* @param node The id of the node * @param id The id of the connection
*/ */
public boolean isConnecting(int node) { public boolean isConnecting(String id) {
NodeConnectionState state = nodeState.get(node); NodeConnectionState state = nodeState.get(id);
return state != null && state.state == ConnectionState.CONNECTING; return state != null && state.state == ConnectionState.CONNECTING;
} }
/** /**
* Enter the connected state for the given node * Enter the connected state for the given connection
* @param node The node we have connected to * @param id The connection identifier
*/ */
public void connected(int node) { public void connected(String id) {
NodeConnectionState nodeState = nodeState(node); NodeConnectionState nodeState = nodeState(id);
nodeState.state = ConnectionState.CONNECTED; nodeState.state = ConnectionState.CONNECTED;
} }
/** /**
* Enter the disconnected state for the given node * Enter the disconnected state for the given node
* @param node The node we have disconnected from * @param id The connection we have disconnected
*/ */
public void disconnected(int node) { public void disconnected(String id) {
NodeConnectionState nodeState = nodeState(node); NodeConnectionState nodeState = nodeState(id);
nodeState.state = ConnectionState.DISCONNECTED; nodeState.state = ConnectionState.DISCONNECTED;
} }
/** /**
* Get the state of our connection to the given node * Get the state of a given connection
* @param node The id of the node * @param id The id of the connection
* @return The state of our connection * @return The state of our connection
*/ */
public ConnectionState connectionState(int node) { public ConnectionState connectionState(String id) {
return nodeState(node).state; return nodeState(id).state;
} }
/** /**
* Get the state of a given node * Get the state of a given node
* @param node The node to fetch the state for * @param id The connection to fetch the state for
*/ */
private NodeConnectionState nodeState(int node) { private NodeConnectionState nodeState(String id) {
NodeConnectionState state = this.nodeState.get(node); NodeConnectionState state = this.nodeState.get(id);
if (state == null) if (state == null)
throw new IllegalStateException("No entry found for node " + node); throw new IllegalStateException("No entry found for connection " + id);
return state; return state;
} }

View File

@ -55,4 +55,6 @@ public class CommonClientConfigs {
public static final String METRIC_REPORTER_CLASSES_CONFIG = "metric.reporters"; public static final String METRIC_REPORTER_CLASSES_CONFIG = "metric.reporters";
public static final String METRIC_REPORTER_CLASSES_DOC = "A list of classes to use as metrics reporters. Implementing the <code>MetricReporter</code> interface allows plugging in classes that will be notified of new metric creation. The JmxReporter is always included to register JMX statistics."; public static final String METRIC_REPORTER_CLASSES_DOC = "A list of classes to use as metrics reporters. Implementing the <code>MetricReporter</code> interface allows plugging in classes that will be notified of new metric creation. The JmxReporter is always included to register JMX statistics.";
public static final String CONNECTIONS_MAX_IDLE_MS_CONFIG = "connections.max.idle.ms";
public static final String CONNECTIONS_MAX_IDLE_MS_DOC = "Close idle connections after the number of milliseconds specified by this config.";
} }

View File

@ -24,14 +24,14 @@ import java.util.Map;
final class InFlightRequests { final class InFlightRequests {
private final int maxInFlightRequestsPerConnection; private final int maxInFlightRequestsPerConnection;
private final Map<Integer, Deque<ClientRequest>> requests = new HashMap<Integer, Deque<ClientRequest>>(); private final Map<String, Deque<ClientRequest>> requests = new HashMap<String, Deque<ClientRequest>>();
public InFlightRequests(int maxInFlightRequestsPerConnection) { public InFlightRequests(int maxInFlightRequestsPerConnection) {
this.maxInFlightRequestsPerConnection = maxInFlightRequestsPerConnection; this.maxInFlightRequestsPerConnection = maxInFlightRequestsPerConnection;
} }
/** /**
* Add the given request to the queue for the node it was directed to * Add the given request to the queue for the connection it was directed to
*/ */
public void add(ClientRequest request) { public void add(ClientRequest request) {
Deque<ClientRequest> reqs = this.requests.get(request.request().destination()); Deque<ClientRequest> reqs = this.requests.get(request.request().destination());
@ -45,7 +45,7 @@ final class InFlightRequests {
/** /**
* Get the request queue for the given node * Get the request queue for the given node
*/ */
private Deque<ClientRequest> requestQueue(int node) { private Deque<ClientRequest> requestQueue(String node) {
Deque<ClientRequest> reqs = requests.get(node); Deque<ClientRequest> reqs = requests.get(node);
if (reqs == null || reqs.isEmpty()) if (reqs == null || reqs.isEmpty())
throw new IllegalStateException("Response from server for which there are no in-flight requests."); throw new IllegalStateException("Response from server for which there are no in-flight requests.");
@ -55,7 +55,7 @@ final class InFlightRequests {
/** /**
* Get the oldest request (the one that that will be completed next) for the given node * Get the oldest request (the one that that will be completed next) for the given node
*/ */
public ClientRequest completeNext(int node) { public ClientRequest completeNext(String node) {
return requestQueue(node).pollLast(); return requestQueue(node).pollLast();
} }
@ -63,7 +63,7 @@ final class InFlightRequests {
* Get the last request we sent to the given node (but don't remove it from the queue) * Get the last request we sent to the given node (but don't remove it from the queue)
* @param node The node id * @param node The node id
*/ */
public ClientRequest lastSent(int node) { public ClientRequest lastSent(String node) {
return requestQueue(node).peekFirst(); return requestQueue(node).peekFirst();
} }
@ -72,7 +72,7 @@ final class InFlightRequests {
* @param node The node the request was sent to * @param node The node the request was sent to
* @return The request * @return The request
*/ */
public ClientRequest completeLastSent(int node) { public ClientRequest completeLastSent(String node) {
return requestQueue(node).pollFirst(); return requestQueue(node).pollFirst();
} }
@ -82,7 +82,7 @@ final class InFlightRequests {
* @param node Node in question * @param node Node in question
* @return true iff we have no requests still being sent to the given node * @return true iff we have no requests still being sent to the given node
*/ */
public boolean canSendMore(int node) { public boolean canSendMore(String node) {
Deque<ClientRequest> queue = requests.get(node); Deque<ClientRequest> queue = requests.get(node);
return queue == null || queue.isEmpty() || return queue == null || queue.isEmpty() ||
(queue.peekFirst().request().completed() && queue.size() < this.maxInFlightRequestsPerConnection); (queue.peekFirst().request().completed() && queue.size() < this.maxInFlightRequestsPerConnection);
@ -93,7 +93,7 @@ final class InFlightRequests {
* @param node The node * @param node The node
* @return The request count. * @return The request count.
*/ */
public int inFlightRequestCount(int node) { public int inFlightRequestCount(String node) {
Deque<ClientRequest> queue = requests.get(node); Deque<ClientRequest> queue = requests.get(node);
return queue == null ? 0 : queue.size(); return queue == null ? 0 : queue.size();
} }
@ -114,7 +114,7 @@ final class InFlightRequests {
* @param node The node * @param node The node
* @return All the in-flight requests for that node that have been removed * @return All the in-flight requests for that node that have been removed
*/ */
public Iterable<ClientRequest> clearAll(int node) { public Iterable<ClientRequest> clearAll(String node) {
Deque<ClientRequest> reqs = requests.get(node); Deque<ClientRequest> reqs = requests.get(node);
if (reqs == null) { if (reqs == null) {
return Collections.emptyList(); return Collections.emptyList();

View File

@ -12,6 +12,7 @@
*/ */
package org.apache.kafka.clients; package org.apache.kafka.clients;
import java.io.Closeable;
import java.util.List; import java.util.List;
import org.apache.kafka.common.Node; import org.apache.kafka.common.Node;
@ -21,7 +22,7 @@ import org.apache.kafka.common.requests.RequestHeader;
/** /**
* The interface for {@link NetworkClient} * The interface for {@link NetworkClient}
*/ */
public interface KafkaClient { public interface KafkaClient extends Closeable {
/** /**
* Check if we are currently ready to send another request to the given node but don't attempt to connect if we * Check if we are currently ready to send another request to the given node but don't attempt to connect if we
@ -80,13 +81,13 @@ public interface KafkaClient {
public List<ClientResponse> poll(long timeout, long now); public List<ClientResponse> poll(long timeout, long now);
/** /**
* Complete all in-flight requests for a given node * Complete all in-flight requests for a given connection
* *
* @param node The node to complete requests for * @param id The connection to complete requests for
* @param now The current time in ms * @param now The current time in ms
* @return All requests that complete during this time period. * @return All requests that complete during this time period.
*/ */
public List<ClientResponse> completeAll(int node, long now); public List<ClientResponse> completeAll(String id, long now);
/** /**
* Complete all in-flight requests * Complete all in-flight requests
@ -116,7 +117,7 @@ public interface KafkaClient {
* *
* @param nodeId The id of the node * @param nodeId The id of the node
*/ */
public int inFlightRequestCount(int nodeId); public int inFlightRequestCount(String nodeId);
/** /**
* Generate a request header for the next request * Generate a request header for the next request
@ -130,9 +131,4 @@ public interface KafkaClient {
*/ */
public void wakeup(); public void wakeup();
/**
* Close the client and disconnect from all nodes
*/
public void close();
} }

View File

@ -36,6 +36,7 @@ public final class Metadata {
private final long metadataExpireMs; private final long metadataExpireMs;
private int version; private int version;
private long lastRefreshMs; private long lastRefreshMs;
private long lastSuccessfulRefreshMs;
private Cluster cluster; private Cluster cluster;
private boolean needUpdate; private boolean needUpdate;
private final Set<String> topics; private final Set<String> topics;
@ -57,6 +58,7 @@ public final class Metadata {
this.refreshBackoffMs = refreshBackoffMs; this.refreshBackoffMs = refreshBackoffMs;
this.metadataExpireMs = metadataExpireMs; this.metadataExpireMs = metadataExpireMs;
this.lastRefreshMs = 0L; this.lastRefreshMs = 0L;
this.lastSuccessfulRefreshMs = 0L;
this.version = 0; this.version = 0;
this.cluster = Cluster.empty(); this.cluster = Cluster.empty();
this.needUpdate = false; this.needUpdate = false;
@ -83,7 +85,7 @@ public final class Metadata {
* is now * is now
*/ */
public synchronized long timeToNextUpdate(long nowMs) { public synchronized long timeToNextUpdate(long nowMs) {
long timeToExpire = needUpdate ? 0 : Math.max(this.lastRefreshMs + this.metadataExpireMs - nowMs, 0); long timeToExpire = needUpdate ? 0 : Math.max(this.lastSuccessfulRefreshMs + this.metadataExpireMs - nowMs, 0);
long timeToAllowUpdate = this.lastRefreshMs + this.refreshBackoffMs - nowMs; long timeToAllowUpdate = this.lastRefreshMs + this.refreshBackoffMs - nowMs;
return Math.max(timeToExpire, timeToAllowUpdate); return Math.max(timeToExpire, timeToAllowUpdate);
} }
@ -146,6 +148,7 @@ public final class Metadata {
public synchronized void update(Cluster cluster, long now) { public synchronized void update(Cluster cluster, long now) {
this.needUpdate = false; this.needUpdate = false;
this.lastRefreshMs = now; this.lastRefreshMs = now;
this.lastSuccessfulRefreshMs = now;
this.version += 1; this.version += 1;
this.cluster = cluster; this.cluster = cluster;
notifyAll(); notifyAll();
@ -168,10 +171,10 @@ public final class Metadata {
} }
/** /**
* The last time metadata was updated. * The last time metadata was successfully updated.
*/ */
public synchronized long lastUpdate() { public synchronized long lastSuccessfulUpdate() {
return this.lastRefreshMs; return this.lastSuccessfulRefreshMs;
} }
/** /**

View File

@ -22,8 +22,8 @@ import java.util.Set;
import org.apache.kafka.common.Cluster; import org.apache.kafka.common.Cluster;
import org.apache.kafka.common.Node; import org.apache.kafka.common.Node;
import org.apache.kafka.common.network.NetworkReceive; import org.apache.kafka.common.network.NetworkReceive;
import org.apache.kafka.common.network.NetworkSend;
import org.apache.kafka.common.network.Selectable; import org.apache.kafka.common.network.Selectable;
import org.apache.kafka.common.network.Send;
import org.apache.kafka.common.protocol.ApiKeys; import org.apache.kafka.common.protocol.ApiKeys;
import org.apache.kafka.common.protocol.ProtoUtils; import org.apache.kafka.common.protocol.ProtoUtils;
import org.apache.kafka.common.protocol.types.Struct; import org.apache.kafka.common.protocol.types.Struct;
@ -111,7 +111,7 @@ public class NetworkClient implements KafkaClient {
if (isReady(node, now)) if (isReady(node, now))
return true; return true;
if (connectionStates.canConnect(node.id(), now)) if (connectionStates.canConnect(node.idString(), now))
// if we are interested in sending to a node and we don't have a connection to it, initiate one // if we are interested in sending to a node and we don't have a connection to it, initiate one
initiateConnect(node, now); initiateConnect(node, now);
@ -129,7 +129,7 @@ public class NetworkClient implements KafkaClient {
*/ */
@Override @Override
public long connectionDelay(Node node, long now) { public long connectionDelay(Node node, long now) {
return connectionStates.connectionDelay(node.id(), now); return connectionStates.connectionDelay(node.idString(), now);
} }
/** /**
@ -142,7 +142,7 @@ public class NetworkClient implements KafkaClient {
*/ */
@Override @Override
public boolean connectionFailed(Node node) { public boolean connectionFailed(Node node) {
return connectionStates.connectionState(node.id()).equals(ConnectionState.DISCONNECTED); return connectionStates.connectionState(node.idString()).equals(ConnectionState.DISCONNECTED);
} }
/** /**
@ -154,7 +154,7 @@ public class NetworkClient implements KafkaClient {
*/ */
@Override @Override
public boolean isReady(Node node, long now) { public boolean isReady(Node node, long now) {
int nodeId = node.id(); String nodeId = node.idString();
if (!this.metadataFetchInProgress && this.metadata.timeToNextUpdate(now) == 0) if (!this.metadataFetchInProgress && this.metadata.timeToNextUpdate(now) == 0)
// if we need to update our metadata now declare all requests unready to make metadata requests first // if we need to update our metadata now declare all requests unready to make metadata requests first
// priority // priority
@ -165,11 +165,11 @@ public class NetworkClient implements KafkaClient {
} }
/** /**
* Are we connected and ready and able to send more requests to the given node? * Are we connected and ready and able to send more requests to the given connection?
* *
* @param node The node * @param node The node
*/ */
private boolean isSendable(int node) { private boolean isSendable(String node) {
return connectionStates.isConnected(node) && inFlightRequests.canSendMore(node); return connectionStates.isConnected(node) && inFlightRequests.canSendMore(node);
} }
@ -179,7 +179,7 @@ public class NetworkClient implements KafkaClient {
* @param node The node to check * @param node The node to check
* @return The connection state * @return The connection state
*/ */
public ConnectionState connectionState(int node) { public ConnectionState connectionState(String node) {
return connectionStates.connectionState(node); return connectionStates.connectionState(node);
} }
@ -190,7 +190,7 @@ public class NetworkClient implements KafkaClient {
*/ */
@Override @Override
public void send(ClientRequest request) { public void send(ClientRequest request) {
int nodeId = request.request().destination(); String nodeId = request.request().destination();
if (!isSendable(nodeId)) if (!isSendable(nodeId))
throw new IllegalStateException("Attempt to send a request to node " + nodeId + " which is not ready."); throw new IllegalStateException("Attempt to send a request to node " + nodeId + " which is not ready.");
@ -252,7 +252,7 @@ public class NetworkClient implements KafkaClient {
* @return All the collected responses * @return All the collected responses
*/ */
@Override @Override
public List<ClientResponse> completeAll(int node, long now) { public List<ClientResponse> completeAll(String node, long now) {
try { try {
this.selector.muteAll(); this.selector.muteAll();
this.selector.unmute(node); this.selector.unmute(node);
@ -288,8 +288,8 @@ public class NetworkClient implements KafkaClient {
* Get the number of in-flight requests for a given node * Get the number of in-flight requests for a given node
*/ */
@Override @Override
public int inFlightRequestCount(int nodeId) { public int inFlightRequestCount(String node) {
return this.inFlightRequests.inFlightRequestCount(nodeId); return this.inFlightRequests.inFlightRequestCount(node);
} }
/** /**
@ -334,11 +334,11 @@ public class NetworkClient implements KafkaClient {
for (int i = 0; i < nodes.size(); i++) { for (int i = 0; i < nodes.size(); i++) {
int idx = Utils.abs((this.nodeIndexOffset + i) % nodes.size()); int idx = Utils.abs((this.nodeIndexOffset + i) % nodes.size());
Node node = nodes.get(idx); Node node = nodes.get(idx);
int currInflight = this.inFlightRequests.inFlightRequestCount(node.id()); int currInflight = this.inFlightRequests.inFlightRequestCount(node.idString());
if (currInflight == 0 && this.connectionStates.isConnected(node.id())) { if (currInflight == 0 && this.connectionStates.isConnected(node.idString())) {
// if we find an established connection with no in-flight requests we can stop right away // if we find an established connection with no in-flight requests we can stop right away
return node; return node;
} else if (!this.connectionStates.isBlackedOut(node.id(), now) && currInflight < inflight) { } else if (!this.connectionStates.isBlackedOut(node.idString(), now) && currInflight < inflight) {
// otherwise if this is the best we have found so far, record that // otherwise if this is the best we have found so far, record that
inflight = currInflight; inflight = currInflight;
found = node; found = node;
@ -355,7 +355,7 @@ public class NetworkClient implements KafkaClient {
*/ */
private void handleCompletedSends(List<ClientResponse> responses, long now) { private void handleCompletedSends(List<ClientResponse> responses, long now) {
// if no response is expected then when the send is completed, return it // if no response is expected then when the send is completed, return it
for (NetworkSend send : this.selector.completedSends()) { for (Send send : this.selector.completedSends()) {
ClientRequest request = this.inFlightRequests.lastSent(send.destination()); ClientRequest request = this.inFlightRequests.lastSent(send.destination());
if (!request.expectResponse()) { if (!request.expectResponse()) {
this.inFlightRequests.completeLastSent(send.destination()); this.inFlightRequests.completeLastSent(send.destination());
@ -372,7 +372,7 @@ public class NetworkClient implements KafkaClient {
*/ */
private void handleCompletedReceives(List<ClientResponse> responses, long now) { private void handleCompletedReceives(List<ClientResponse> responses, long now) {
for (NetworkReceive receive : this.selector.completedReceives()) { for (NetworkReceive receive : this.selector.completedReceives()) {
int source = receive.source(); String source = receive.source();
ClientRequest req = inFlightRequests.completeNext(source); ClientRequest req = inFlightRequests.completeNext(source);
ResponseHeader header = ResponseHeader.parse(receive.payload()); ResponseHeader header = ResponseHeader.parse(receive.payload());
short apiKey = req.request().header().apiKey(); short apiKey = req.request().header().apiKey();
@ -391,6 +391,10 @@ public class NetworkClient implements KafkaClient {
this.metadataFetchInProgress = false; this.metadataFetchInProgress = false;
MetadataResponse response = new MetadataResponse(body); MetadataResponse response = new MetadataResponse(body);
Cluster cluster = response.cluster(); Cluster cluster = response.cluster();
// check if any topics metadata failed to get updated
if (response.errors().size() > 0) {
log.warn("Error while fetching metadata with correlation id {} : {}", header.correlationId(), response.errors());
}
// don't update the cluster if there are no valid nodes...the topic we want may still be in the process of being // don't update the cluster if there are no valid nodes...the topic we want may still be in the process of being
// created which means we will get errors and no nodes until it exists // created which means we will get errors and no nodes until it exists
if (cluster.nodes().size() > 0) { if (cluster.nodes().size() > 0) {
@ -408,7 +412,7 @@ public class NetworkClient implements KafkaClient {
* @param now The current time * @param now The current time
*/ */
private void handleDisconnections(List<ClientResponse> responses, long now) { private void handleDisconnections(List<ClientResponse> responses, long now) {
for (int node : this.selector.disconnected()) { for (String node : this.selector.disconnected()) {
connectionStates.disconnected(node); connectionStates.disconnected(node);
log.debug("Node {} disconnected.", node); log.debug("Node {} disconnected.", node);
for (ClientRequest request : this.inFlightRequests.clearAll(node)) { for (ClientRequest request : this.inFlightRequests.clearAll(node)) {
@ -429,9 +433,9 @@ public class NetworkClient implements KafkaClient {
* Record any newly completed connections * Record any newly completed connections
*/ */
private void handleConnections() { private void handleConnections() {
for (Integer id : this.selector.connected()) { for (String node : this.selector.connected()) {
log.debug("Completed connection to node {}", id); log.debug("Completed connection to node {}", node);
this.connectionStates.connected(id); this.connectionStates.connected(node);
} }
} }
@ -447,7 +451,7 @@ public class NetworkClient implements KafkaClient {
/** /**
* Create a metadata request for the given topics * Create a metadata request for the given topics
*/ */
private ClientRequest metadataRequest(long now, int node, Set<String> topics) { private ClientRequest metadataRequest(long now, String node, Set<String> topics) {
MetadataRequest metadata = new MetadataRequest(new ArrayList<String>(topics)); MetadataRequest metadata = new MetadataRequest(new ArrayList<String>(topics));
RequestSend send = new RequestSend(node, nextRequestHeader(ApiKeys.METADATA), metadata.toStruct()); RequestSend send = new RequestSend(node, nextRequestHeader(ApiKeys.METADATA), metadata.toStruct());
return new ClientRequest(now, true, send, null); return new ClientRequest(now, true, send, null);
@ -466,15 +470,17 @@ public class NetworkClient implements KafkaClient {
this.lastNoNodeAvailableMs = now; this.lastNoNodeAvailableMs = now;
return; return;
} }
String nodeConnectionId = node.idString();
if (connectionStates.isConnected(node.id()) && inFlightRequests.canSendMore(node.id())) {
if (connectionStates.isConnected(nodeConnectionId) && inFlightRequests.canSendMore(nodeConnectionId)) {
Set<String> topics = metadata.topics(); Set<String> topics = metadata.topics();
this.metadataFetchInProgress = true; this.metadataFetchInProgress = true;
ClientRequest metadataRequest = metadataRequest(now, node.id(), topics); ClientRequest metadataRequest = metadataRequest(now, nodeConnectionId, topics);
log.debug("Sending metadata request {} to node {}", metadataRequest, node.id()); log.debug("Sending metadata request {} to node {}", metadataRequest, node.id());
this.selector.send(metadataRequest.request()); this.selector.send(metadataRequest.request());
this.inFlightRequests.add(metadataRequest); this.inFlightRequests.add(metadataRequest);
} else if (connectionStates.canConnect(node.id(), now)) { } else if (connectionStates.canConnect(nodeConnectionId, now)) {
// we don't have a connection to this node right now, make one // we don't have a connection to this node right now, make one
log.debug("Initialize connection to node {} for sending metadata request", node.id()); log.debug("Initialize connection to node {} for sending metadata request", node.id());
initiateConnect(node, now); initiateConnect(node, now);
@ -493,16 +499,17 @@ public class NetworkClient implements KafkaClient {
* Initiate a connection to the given node * Initiate a connection to the given node
*/ */
private void initiateConnect(Node node, long now) { private void initiateConnect(Node node, long now) {
String nodeConnectionId = node.idString();
try { try {
log.debug("Initiating connection to node {} at {}:{}.", node.id(), node.host(), node.port()); log.debug("Initiating connection to node {} at {}:{}.", node.id(), node.host(), node.port());
this.connectionStates.connecting(node.id(), now); this.connectionStates.connecting(nodeConnectionId, now);
selector.connect(node.id(), selector.connect(nodeConnectionId,
new InetSocketAddress(node.host(), node.port()), new InetSocketAddress(node.host(), node.port()),
this.socketSendBuffer, this.socketSendBuffer,
this.socketReceiveBuffer); this.socketReceiveBuffer);
} catch (IOException e) { } catch (IOException e) {
/* attempt failed, we'll try again after the backoff */ /* attempt failed, we'll try again after the backoff */
connectionStates.disconnected(node.id()); connectionStates.disconnected(nodeConnectionId);
/* maybe the problem is our metadata, update it */ /* maybe the problem is our metadata, update it */
metadata.requestUpdate(); metadata.requestUpdate();
log.debug("Error connecting to node {} at {}:{}:", node.id(), node.host(), node.port(), e); log.debug("Error connecting to node {} at {}:{}:", node.id(), node.host(), node.port(), e);

View File

@ -12,13 +12,6 @@
*/ */
package org.apache.kafka.clients.consumer; package org.apache.kafka.clients.consumer;
import static org.apache.kafka.common.config.ConfigDef.Range.atLeast;
import static org.apache.kafka.common.config.ConfigDef.ValidString.in;
import java.util.HashMap;
import java.util.Map;
import java.util.Properties;
import org.apache.kafka.clients.CommonClientConfigs; import org.apache.kafka.clients.CommonClientConfigs;
import org.apache.kafka.clients.consumer.internals.NoOpConsumerRebalanceCallback; import org.apache.kafka.clients.consumer.internals.NoOpConsumerRebalanceCallback;
import org.apache.kafka.common.config.AbstractConfig; import org.apache.kafka.common.config.AbstractConfig;
@ -27,6 +20,13 @@ import org.apache.kafka.common.config.ConfigDef.Importance;
import org.apache.kafka.common.config.ConfigDef.Type; import org.apache.kafka.common.config.ConfigDef.Type;
import org.apache.kafka.common.serialization.Deserializer; import org.apache.kafka.common.serialization.Deserializer;
import java.util.HashMap;
import java.util.Map;
import java.util.Properties;
import static org.apache.kafka.common.config.ConfigDef.Range.atLeast;
import static org.apache.kafka.common.config.ConfigDef.ValidString.in;
/** /**
* The consumer configuration keys * The consumer configuration keys
*/ */
@ -156,6 +156,9 @@ public class ConsumerConfig extends AbstractConfig {
public static final String VALUE_DESERIALIZER_CLASS_CONFIG = "value.deserializer"; public static final String VALUE_DESERIALIZER_CLASS_CONFIG = "value.deserializer";
private static final String VALUE_DESERIALIZER_CLASS_DOC = "Deserializer class for value that implements the <code>Deserializer</code> interface."; private static final String VALUE_DESERIALIZER_CLASS_DOC = "Deserializer class for value that implements the <code>Deserializer</code> interface.";
/** <code>connections.max.idle.ms</code> */
public static final String CONNECTIONS_MAX_IDLE_MS_CONFIG = CommonClientConfigs.CONNECTIONS_MAX_IDLE_MS_CONFIG;
static { static {
CONFIG = new ConfigDef().define(BOOTSTRAP_SERVERS_CONFIG, CONFIG = new ConfigDef().define(BOOTSTRAP_SERVERS_CONFIG,
@ -277,7 +280,13 @@ public class ConsumerConfig extends AbstractConfig {
.define(VALUE_DESERIALIZER_CLASS_CONFIG, .define(VALUE_DESERIALIZER_CLASS_CONFIG,
Type.CLASS, Type.CLASS,
Importance.HIGH, Importance.HIGH,
VALUE_DESERIALIZER_CLASS_DOC); VALUE_DESERIALIZER_CLASS_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,
9 * 60 * 1000,
Importance.MEDIUM,
CommonClientConfigs.CONNECTIONS_MAX_IDLE_MS_DOC);
} }
public static Map<String, Object> addDeserializerToConfig(Map<String, Object> configs, public static Map<String, Object> addDeserializerToConfig(Map<String, Object> configs,
@ -304,7 +313,7 @@ public class ConsumerConfig extends AbstractConfig {
return newProperties; return newProperties;
} }
ConsumerConfig(Map<? extends Object, ? extends Object> props) { ConsumerConfig(Map<?, ?> props) {
super(CONFIG, props); super(CONFIG, props);
} }

View File

@ -58,14 +58,14 @@ public final class ConsumerRecord<K, V> {
/** /**
* The key (or null if no key is specified) * The key (or null if no key is specified)
*/ */
public K key() throws Exception { public K key() {
return key; return key;
} }
/** /**
* The value * The value
*/ */
public V value() throws Exception { public V value() {
return value; return value;
} }

View File

@ -12,15 +12,15 @@
*/ */
package org.apache.kafka.clients.consumer; package org.apache.kafka.clients.consumer;
import org.apache.kafka.common.TopicPartition;
import org.apache.kafka.common.utils.AbstractIterator;
import java.util.ArrayList; import java.util.ArrayList;
import java.util.Collections; import java.util.Collections;
import java.util.Iterator; import java.util.Iterator;
import java.util.List; import java.util.List;
import java.util.Map; import java.util.Map;
import org.apache.kafka.common.TopicPartition;
import org.apache.kafka.common.utils.AbstractIterator;
/** /**
* A container that holds the list {@link ConsumerRecord} per partition for a * A container that holds the list {@link ConsumerRecord} per partition for a
* particular topic. There is one for every topic returned by a * particular topic. There is one for every topic returned by a
@ -55,7 +55,7 @@ public class ConsumerRecords<K, V> implements Iterable<ConsumerRecord<K, V>> {
throw new IllegalArgumentException("Topic must be non-null."); throw new IllegalArgumentException("Topic must be non-null.");
List<List<ConsumerRecord<K, V>>> recs = new ArrayList<List<ConsumerRecord<K, V>>>(); List<List<ConsumerRecord<K, V>>> recs = new ArrayList<List<ConsumerRecord<K, V>>>();
for (Map.Entry<TopicPartition, List<ConsumerRecord<K, V>>> entry : records.entrySet()) { for (Map.Entry<TopicPartition, List<ConsumerRecord<K, V>>> entry : records.entrySet()) {
if (entry.getKey().equals(topic)) if (entry.getKey().topic().equals(topic))
recs.add(entry.getValue()); recs.add(entry.getValue());
} }
return new ConcatenatedIterable<K, V>(recs); return new ConcatenatedIterable<K, V>(recs);

View File

@ -24,6 +24,7 @@ import java.util.Properties;
import java.util.Set; import java.util.Set;
import java.util.concurrent.TimeUnit; import java.util.concurrent.TimeUnit;
import java.util.concurrent.atomic.AtomicInteger; import java.util.concurrent.atomic.AtomicInteger;
import java.util.concurrent.atomic.AtomicReference;
import org.apache.kafka.clients.ClientUtils; import org.apache.kafka.clients.ClientUtils;
import org.apache.kafka.clients.Metadata; import org.apache.kafka.clients.Metadata;
@ -32,6 +33,7 @@ import org.apache.kafka.clients.consumer.internals.Coordinator;
import org.apache.kafka.clients.consumer.internals.Fetcher; import org.apache.kafka.clients.consumer.internals.Fetcher;
import org.apache.kafka.clients.consumer.internals.SubscriptionState; import org.apache.kafka.clients.consumer.internals.SubscriptionState;
import org.apache.kafka.common.Cluster; import org.apache.kafka.common.Cluster;
import org.apache.kafka.common.KafkaException;
import org.apache.kafka.common.Metric; import org.apache.kafka.common.Metric;
import org.apache.kafka.common.PartitionInfo; import org.apache.kafka.common.PartitionInfo;
import org.apache.kafka.common.TopicPartition; import org.apache.kafka.common.TopicPartition;
@ -346,6 +348,8 @@ public class KafkaConsumer<K, V> implements Consumer<K, V> {
private static final AtomicInteger CONSUMER_CLIENT_ID_SEQUENCE = new AtomicInteger(1); private static final AtomicInteger CONSUMER_CLIENT_ID_SEQUENCE = new AtomicInteger(1);
private final Coordinator coordinator; private final Coordinator coordinator;
private final Deserializer<K> keyDeserializer;
private final Deserializer<V> valueDeserializer;
private final Fetcher<K, V> fetcher; private final Fetcher<K, V> fetcher;
private final Time time; private final Time time;
@ -437,74 +441,98 @@ public class KafkaConsumer<K, V> implements Consumer<K, V> {
ConsumerRebalanceCallback callback, ConsumerRebalanceCallback callback,
Deserializer<K> keyDeserializer, Deserializer<K> keyDeserializer,
Deserializer<V> valueDeserializer) { Deserializer<V> valueDeserializer) {
log.debug("Starting the Kafka consumer"); try {
if (callback == null) log.debug("Starting the Kafka consumer");
this.rebalanceCallback = config.getConfiguredInstance(ConsumerConfig.CONSUMER_REBALANCE_CALLBACK_CLASS_CONFIG, if (callback == null)
ConsumerRebalanceCallback.class); this.rebalanceCallback = config.getConfiguredInstance(ConsumerConfig.CONSUMER_REBALANCE_CALLBACK_CLASS_CONFIG,
else ConsumerRebalanceCallback.class);
this.rebalanceCallback = callback; else
this.time = new SystemTime(); this.rebalanceCallback = callback;
this.autoCommit = config.getBoolean(ConsumerConfig.ENABLE_AUTO_COMMIT_CONFIG); this.time = new SystemTime();
this.autoCommitIntervalMs = config.getLong(ConsumerConfig.AUTO_COMMIT_INTERVAL_MS_CONFIG); this.autoCommit = config.getBoolean(ConsumerConfig.ENABLE_AUTO_COMMIT_CONFIG);
this.lastCommitAttemptMs = time.milliseconds(); this.autoCommitIntervalMs = config.getLong(ConsumerConfig.AUTO_COMMIT_INTERVAL_MS_CONFIG);
this.lastCommitAttemptMs = time.milliseconds();
MetricConfig metricConfig = new MetricConfig().samples(config.getInt(ConsumerConfig.METRICS_NUM_SAMPLES_CONFIG)) MetricConfig metricConfig = new MetricConfig().samples(config.getInt(ConsumerConfig.METRICS_NUM_SAMPLES_CONFIG))
.timeWindow(config.getLong(ConsumerConfig.METRICS_SAMPLE_WINDOW_MS_CONFIG), .timeWindow(config.getLong(ConsumerConfig.METRICS_SAMPLE_WINDOW_MS_CONFIG),
TimeUnit.MILLISECONDS); TimeUnit.MILLISECONDS);
String clientId = config.getString(ConsumerConfig.CLIENT_ID_CONFIG); String clientId = config.getString(ConsumerConfig.CLIENT_ID_CONFIG);
String jmxPrefix = "kafka.consumer"; String jmxPrefix = "kafka.consumer";
if (clientId.length() <= 0) if (clientId.length() <= 0)
clientId = "consumer-" + CONSUMER_CLIENT_ID_SEQUENCE.getAndIncrement(); clientId = "consumer-" + CONSUMER_CLIENT_ID_SEQUENCE.getAndIncrement();
List<MetricsReporter> reporters = config.getConfiguredInstances(ConsumerConfig.METRIC_REPORTER_CLASSES_CONFIG, List<MetricsReporter> reporters = config.getConfiguredInstances(ConsumerConfig.METRIC_REPORTER_CLASSES_CONFIG,
MetricsReporter.class); MetricsReporter.class);
reporters.add(new JmxReporter(jmxPrefix)); reporters.add(new JmxReporter(jmxPrefix));
this.metrics = new Metrics(metricConfig, reporters, time); this.metrics = new Metrics(metricConfig, reporters, time);
this.retryBackoffMs = config.getLong(ConsumerConfig.RETRY_BACKOFF_MS_CONFIG); this.retryBackoffMs = config.getLong(ConsumerConfig.RETRY_BACKOFF_MS_CONFIG);
this.metadata = new Metadata(retryBackoffMs, config.getLong(ConsumerConfig.METADATA_MAX_AGE_CONFIG)); this.metadata = new Metadata(retryBackoffMs, config.getLong(ConsumerConfig.METADATA_MAX_AGE_CONFIG));
List<InetSocketAddress> addresses = ClientUtils.parseAndValidateAddresses(config.getList(ConsumerConfig.BOOTSTRAP_SERVERS_CONFIG)); List<InetSocketAddress> addresses = ClientUtils.parseAndValidateAddresses(config.getList(ConsumerConfig.BOOTSTRAP_SERVERS_CONFIG));
this.metadata.update(Cluster.bootstrap(addresses), 0); this.metadata.update(Cluster.bootstrap(addresses), 0);
String metricGrpPrefix = "consumer"; String metricGrpPrefix = "consumer";
Map<String, String> metricsTags = new LinkedHashMap<String, String>(); Map<String, String> metricsTags = new LinkedHashMap<String, String>();
metricsTags.put("client-id", clientId); metricsTags.put("client-id", clientId);
this.client = new NetworkClient(new Selector(metrics, time, metricGrpPrefix, metricsTags), this.client = new NetworkClient(
this.metadata, new Selector(config.getLong(ConsumerConfig.CONNECTIONS_MAX_IDLE_MS_CONFIG), metrics, time, metricGrpPrefix, metricsTags),
clientId, this.metadata,
100, // a fixed large enough value will suffice clientId,
config.getLong(ConsumerConfig.RECONNECT_BACKOFF_MS_CONFIG), 100, // a fixed large enough value will suffice
config.getInt(ConsumerConfig.SEND_BUFFER_CONFIG), config.getLong(ConsumerConfig.RECONNECT_BACKOFF_MS_CONFIG),
config.getInt(ConsumerConfig.RECEIVE_BUFFER_CONFIG)); config.getInt(ConsumerConfig.SEND_BUFFER_CONFIG),
this.subscriptions = new SubscriptionState(); config.getInt(ConsumerConfig.RECEIVE_BUFFER_CONFIG));
this.coordinator = new Coordinator(this.client, this.subscriptions = new SubscriptionState();
config.getString(ConsumerConfig.GROUP_ID_CONFIG), this.coordinator = new Coordinator(this.client,
this.retryBackoffMs, config.getString(ConsumerConfig.GROUP_ID_CONFIG),
config.getLong(ConsumerConfig.SESSION_TIMEOUT_MS_CONFIG), this.retryBackoffMs,
config.getString(ConsumerConfig.PARTITION_ASSIGNMENT_STRATEGY_CONFIG), config.getLong(ConsumerConfig.SESSION_TIMEOUT_MS_CONFIG),
this.metadata, config.getString(ConsumerConfig.PARTITION_ASSIGNMENT_STRATEGY_CONFIG),
this.subscriptions, this.metadata,
metrics, this.subscriptions,
metricGrpPrefix, metrics,
metricsTags, metricGrpPrefix,
this.time); metricsTags,
this.fetcher = new Fetcher<K, V>(this.client, this.time);
this.retryBackoffMs,
config.getInt(ConsumerConfig.FETCH_MIN_BYTES_CONFIG),
config.getInt(ConsumerConfig.FETCH_MAX_WAIT_MS_CONFIG),
config.getInt(ConsumerConfig.MAX_PARTITION_FETCH_BYTES_CONFIG),
config.getBoolean(ConsumerConfig.CHECK_CRCS_CONFIG),
config.getString(ConsumerConfig.AUTO_OFFSET_RESET_CONFIG).toUpperCase(),
keyDeserializer == null ? config.getConfiguredInstance(ConsumerConfig.KEY_DESERIALIZER_CLASS_CONFIG, Deserializer.class) : keyDeserializer,
valueDeserializer == null ? config.getConfiguredInstance(ConsumerConfig.VALUE_DESERIALIZER_CLASS_CONFIG, Deserializer.class) : valueDeserializer,
this.metadata,
this.subscriptions,
metrics,
metricGrpPrefix,
metricsTags,
this.time);
config.logUnused(); if (keyDeserializer == null) {
this.keyDeserializer = config.getConfiguredInstance(ConsumerConfig.KEY_DESERIALIZER_CLASS_CONFIG,
Deserializer.class);
this.keyDeserializer.configure(config.originals(), true);
} else {
this.keyDeserializer = keyDeserializer;
}
if (valueDeserializer == null) {
this.valueDeserializer = config.getConfiguredInstance(ConsumerConfig.VALUE_DESERIALIZER_CLASS_CONFIG,
Deserializer.class);
this.valueDeserializer.configure(config.originals(), false);
} else {
this.valueDeserializer = valueDeserializer;
}
this.fetcher = new Fetcher<K, V>(this.client,
this.retryBackoffMs,
config.getInt(ConsumerConfig.FETCH_MIN_BYTES_CONFIG),
config.getInt(ConsumerConfig.FETCH_MAX_WAIT_MS_CONFIG),
config.getInt(ConsumerConfig.MAX_PARTITION_FETCH_BYTES_CONFIG),
config.getBoolean(ConsumerConfig.CHECK_CRCS_CONFIG),
config.getString(ConsumerConfig.AUTO_OFFSET_RESET_CONFIG).toUpperCase(),
this.keyDeserializer,
this.valueDeserializer,
this.metadata,
this.subscriptions,
metrics,
metricGrpPrefix,
metricsTags,
this.time);
log.debug("Kafka consumer created"); config.logUnused();
log.debug("Kafka consumer created");
} catch (Throwable t) {
// call close methods if internal objects are already constructed
// this is to prevent resource leak. see KAFKA-2121
close(true);
// now propagate the exception
throw new KafkaException("Failed to construct kafka consumer", t);
}
} }
/** /**
@ -806,13 +834,24 @@ public class KafkaConsumer<K, V> implements Consumer<K, V> {
@Override @Override
public synchronized void close() { public synchronized void close() {
log.trace("Closing the Kafka consumer."); close(false);
this.closed = true;
this.metrics.close();
this.client.close();
log.debug("The Kafka consumer has closed.");
} }
private void close(boolean swallowException) {
log.trace("Closing the Kafka consumer.");
AtomicReference<Throwable> firstException = new AtomicReference<Throwable>();
this.closed = true;
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);
log.debug("The Kafka consumer has closed.");
if (firstException.get() != null && !swallowException) {
throw new KafkaException("Failed to close kafka consumer", firstException.get());
}
}
private boolean shouldAutoCommit(long now) { private boolean shouldAutoCommit(long now) {
return this.autoCommit && this.lastCommitAttemptMs <= now - this.autoCommitIntervalMs; return this.autoCommit && this.lastCommitAttemptMs <= now - this.autoCommitIntervalMs;
} }

View File

@ -96,7 +96,7 @@ public final class Coordinator {
this.time = time; this.time = time;
this.client = client; this.client = client;
this.generation = -1; this.generation = -1;
this.consumerId = ""; this.consumerId = JoinGroupRequest.UNKNOWN_CONSUMER_ID;
this.groupId = groupId; this.groupId = groupId;
this.metadata = metadata; this.metadata = metadata;
this.consumerCoordinator = null; this.consumerCoordinator = null;
@ -120,29 +120,58 @@ public final class Coordinator {
// send a join group request to the coordinator // send a join group request to the coordinator
log.debug("(Re-)joining group {} with subscribed topics {}", groupId, subscribedTopics); log.debug("(Re-)joining group {} with subscribed topics {}", groupId, subscribedTopics);
JoinGroupRequest request = new JoinGroupRequest(groupId, // repeat processing the response until succeed or fatal error
(int) this.sessionTimeoutMs, do {
subscribedTopics, JoinGroupRequest request = new JoinGroupRequest(groupId,
this.consumerId, (int) this.sessionTimeoutMs,
this.assignmentStrategy); subscribedTopics,
ClientResponse resp = this.blockingCoordinatorRequest(ApiKeys.JOIN_GROUP, request.toStruct(), null, now); this.consumerId,
this.assignmentStrategy);
// process the response ClientResponse resp = this.blockingCoordinatorRequest(ApiKeys.JOIN_GROUP, request.toStruct(), null, now);
JoinGroupResponse response = new JoinGroupResponse(resp.responseBody()); JoinGroupResponse response = new JoinGroupResponse(resp.responseBody());
// TODO: needs to handle disconnects and errors, should not just throw exceptions short errorCode = response.errorCode();
Errors.forCode(response.errorCode()).maybeThrow();
this.consumerId = response.consumerId();
// set the flag to refresh last committed offsets if (errorCode == Errors.NONE.code()) {
this.subscriptions.needRefreshCommits(); this.consumerId = response.consumerId();
this.generation = response.generationId();
log.debug("Joined group: {}", response); // set the flag to refresh last committed offsets
this.subscriptions.needRefreshCommits();
// record re-assignment time log.debug("Joined group: {}", response);
this.sensors.partitionReassignments.record(time.milliseconds() - now);
// return assigned partitions // record re-assignment time
return response.assignedPartitions(); this.sensors.partitionReassignments.record(time.milliseconds() - now);
// return assigned partitions
return response.assignedPartitions();
} else if (errorCode == Errors.UNKNOWN_CONSUMER_ID.code()) {
// reset the consumer id and retry immediately
this.consumerId = JoinGroupRequest.UNKNOWN_CONSUMER_ID;
log.info("Attempt to join group {} failed due to unknown consumer id, resetting and retrying.",
groupId);
} else if (errorCode == Errors.CONSUMER_COORDINATOR_NOT_AVAILABLE.code()
|| errorCode == Errors.NOT_COORDINATOR_FOR_CONSUMER.code()) {
// re-discover the coordinator and retry with backoff
coordinatorDead();
Utils.sleep(this.retryBackoffMs);
log.info("Attempt to join group {} failed due to obsolete coordinator information, retrying.",
groupId);
} else if (errorCode == Errors.UNKNOWN_PARTITION_ASSIGNMENT_STRATEGY.code()
|| errorCode == Errors.INCONSISTENT_PARTITION_ASSIGNMENT_STRATEGY.code()
|| errorCode == Errors.INVALID_SESSION_TIMEOUT.code()) {
// log the error and re-throw the exception
log.error("Attempt to join group {} failed due to: {}",
groupId, Errors.forCode(errorCode).exception().getMessage());
Errors.forCode(errorCode).maybeThrow();
} else {
// unexpected error, throw the exception
throw new KafkaException("Unexpected error in join group response: "
+ Errors.forCode(response.errorCode()).exception().getMessage());
}
} while (true);
} }
/** /**
@ -216,7 +245,6 @@ public final class Coordinator {
// parse the response to get the offsets // parse the response to get the offsets
boolean offsetsReady = true; boolean offsetsReady = true;
OffsetFetchResponse response = new OffsetFetchResponse(resp.responseBody()); OffsetFetchResponse response = new OffsetFetchResponse(resp.responseBody());
// TODO: needs to handle disconnects
Map<TopicPartition, Long> offsets = new HashMap<TopicPartition, Long>(response.responseData().size()); Map<TopicPartition, Long> offsets = new HashMap<TopicPartition, Long>(response.responseData().size());
for (Map.Entry<TopicPartition, OffsetFetchResponse.PartitionData> entry : response.responseData().entrySet()) { for (Map.Entry<TopicPartition, OffsetFetchResponse.PartitionData> entry : response.responseData().entrySet()) {
TopicPartition tp = entry.getKey(); TopicPartition tp = entry.getKey();
@ -238,7 +266,8 @@ public final class Coordinator {
// just ignore this partition // just ignore this partition
log.debug("Unknown topic or partition for " + tp); log.debug("Unknown topic or partition for " + tp);
} else { } else {
throw new IllegalStateException("Unexpected error code " + data.errorCode + " while fetching offset"); throw new KafkaException("Unexpected error in fetch offset response: "
+ Errors.forCode(data.errorCode).exception().getMessage());
} }
} else if (data.offset >= 0) { } else if (data.offset >= 0) {
// record the position with the offset (-1 indicates no committed offset to fetch) // record the position with the offset (-1 indicates no committed offset to fetch)
@ -416,7 +445,7 @@ public final class Coordinator {
log.debug("Issuing consumer metadata request to broker {}", node.id()); log.debug("Issuing consumer metadata request to broker {}", node.id());
ConsumerMetadataRequest request = new ConsumerMetadataRequest(this.groupId); ConsumerMetadataRequest request = new ConsumerMetadataRequest(this.groupId);
RequestSend send = new RequestSend(node.id(), RequestSend send = new RequestSend(node.idString(),
this.client.nextRequestHeader(ApiKeys.CONSUMER_METADATA), this.client.nextRequestHeader(ApiKeys.CONSUMER_METADATA),
request.toStruct()); request.toStruct());
long now = time.milliseconds(); long now = time.milliseconds();
@ -435,7 +464,7 @@ public final class Coordinator {
log.debug("Issuing request ({}: {}) to coordinator {}", api, request, this.consumerCoordinator.id()); log.debug("Issuing request ({}: {}) to coordinator {}", api, request, this.consumerCoordinator.id());
RequestHeader header = this.client.nextRequestHeader(api); RequestHeader header = this.client.nextRequestHeader(api);
RequestSend send = new RequestSend(this.consumerCoordinator.id(), header, request); RequestSend send = new RequestSend(this.consumerCoordinator.idString(), header, request);
return new ClientRequest(now, true, send, handler); return new ClientRequest(now, true, send, handler);
} }
@ -470,9 +499,15 @@ public final class Coordinator {
if (response.errorCode() == Errors.NONE.code()) { if (response.errorCode() == Errors.NONE.code()) {
log.debug("Received successful heartbeat response."); log.debug("Received successful heartbeat response.");
} else if (response.errorCode() == Errors.CONSUMER_COORDINATOR_NOT_AVAILABLE.code() } else if (response.errorCode() == Errors.CONSUMER_COORDINATOR_NOT_AVAILABLE.code()
|| response.errorCode() == Errors.NOT_COORDINATOR_FOR_CONSUMER.code()) { || response.errorCode() == Errors.NOT_COORDINATOR_FOR_CONSUMER.code()) {
log.info("Attempt to heart beat failed since coordinator is either not started or not valid, marking it as dead.");
coordinatorDead(); coordinatorDead();
} else if (response.errorCode() == Errors.ILLEGAL_GENERATION.code()) { } else if (response.errorCode() == Errors.ILLEGAL_GENERATION.code()) {
log.info("Attempt to heart beat failed since generation id is not legal, try to re-join group.");
subscriptions.needReassignment();
} else if (response.errorCode() == Errors.UNKNOWN_CONSUMER_ID.code()) {
log.info("Attempt to heart beat failed since consumer id is not valid, reset it and try to re-join group.");
consumerId = JoinGroupRequest.UNKNOWN_CONSUMER_ID;
subscriptions.needReassignment(); subscriptions.needReassignment();
} else { } else {
throw new KafkaException("Unexpected error in heartbeat response: " throw new KafkaException("Unexpected error in heartbeat response: "
@ -505,9 +540,10 @@ public final class Coordinator {
log.debug("Committed offset {} for partition {}", offset, tp); log.debug("Committed offset {} for partition {}", offset, tp);
subscriptions.committed(tp, offset); subscriptions.committed(tp, offset);
} else if (errorCode == Errors.CONSUMER_COORDINATOR_NOT_AVAILABLE.code() } else if (errorCode == Errors.CONSUMER_COORDINATOR_NOT_AVAILABLE.code()
|| errorCode == Errors.NOT_COORDINATOR_FOR_CONSUMER.code()) { || errorCode == Errors.NOT_COORDINATOR_FOR_CONSUMER.code()) {
coordinatorDead(); coordinatorDead();
} else { } else {
// do not need to throw the exception but just log the error
log.error("Error committing partition {} at offset {}: {}", log.error("Error committing partition {} at offset {}: {}",
tp, tp,
offset, offset,

View File

@ -124,7 +124,7 @@ public class Fetcher<K, V> {
*/ */
public void initFetches(Cluster cluster, long now) { public void initFetches(Cluster cluster, long now) {
for (ClientRequest request : createFetchRequests(cluster)) { for (ClientRequest request : createFetchRequests(cluster)) {
Node node = cluster.nodeById(request.request().destination()); Node node = cluster.nodeById(Integer.parseInt(request.request().destination()));
if (client.ready(node, now)) { if (client.ready(node, now)) {
log.trace("Initiating fetch to node {}: {}", node.id(), request); log.trace("Initiating fetch to node {}: {}", node.id(), request);
client.send(request); client.send(request);
@ -209,12 +209,12 @@ public class Fetcher<K, V> {
} else if (this.client.ready(info.leader(), now)) { } else if (this.client.ready(info.leader(), now)) {
Node node = info.leader(); Node node = info.leader();
ListOffsetRequest request = new ListOffsetRequest(-1, partitions); ListOffsetRequest request = new ListOffsetRequest(-1, partitions);
RequestSend send = new RequestSend(node.id(), RequestSend send = new RequestSend(node.idString(),
this.client.nextRequestHeader(ApiKeys.LIST_OFFSETS), this.client.nextRequestHeader(ApiKeys.LIST_OFFSETS),
request.toStruct()); request.toStruct());
ClientRequest clientRequest = new ClientRequest(now, true, send, null); ClientRequest clientRequest = new ClientRequest(now, true, send, null);
this.client.send(clientRequest); this.client.send(clientRequest);
List<ClientResponse> responses = this.client.completeAll(node.id(), now); List<ClientResponse> responses = this.client.completeAll(node.idString(), now);
if (responses.isEmpty()) if (responses.isEmpty())
throw new IllegalStateException("This should not happen."); throw new IllegalStateException("This should not happen.");
ClientResponse response = responses.get(responses.size() - 1); ClientResponse response = responses.get(responses.size() - 1);
@ -231,13 +231,14 @@ public class Fetcher<K, V> {
log.debug("Fetched offset {} for partition {}", offset, topicPartition); log.debug("Fetched offset {} for partition {}", offset, topicPartition);
return offset; return offset;
} else if (errorCode == Errors.NOT_LEADER_FOR_PARTITION.code() } else if (errorCode == Errors.NOT_LEADER_FOR_PARTITION.code()
|| errorCode == Errors.UNKNOWN_TOPIC_OR_PARTITION.code()) { || errorCode == Errors.UNKNOWN_TOPIC_OR_PARTITION.code()) {
log.warn("Attempt to fetch offsets for partition {} failed due to obsolete leadership information, retrying.", log.warn("Attempt to fetch offsets for partition {} failed due to obsolete leadership information, retrying.",
topicPartition); topicPartition);
awaitMetadataUpdate(); awaitMetadataUpdate();
} else { } else {
// TODO: we should not just throw exceptions but should handle and log it. log.error("Attempt to fetch offsets for partition {} failed due to: {}",
Errors.forCode(errorCode).maybeThrow(); topicPartition, Errors.forCode(errorCode).exception().getMessage());
awaitMetadataUpdate();
} }
} }
} else { } else {
@ -257,7 +258,7 @@ public class Fetcher<K, V> {
for (TopicPartition partition : subscriptions.assignedPartitions()) { for (TopicPartition partition : subscriptions.assignedPartitions()) {
Node node = cluster.leaderFor(partition); Node node = cluster.leaderFor(partition);
// if there is a leader and no in-flight requests, issue a new fetch // if there is a leader and no in-flight requests, issue a new fetch
if (node != null && this.client.inFlightRequestCount(node.id()) == 0) { if (node != null && this.client.inFlightRequestCount(node.idString()) == 0) {
Map<TopicPartition, FetchRequest.PartitionData> fetch = fetchable.get(node.id()); Map<TopicPartition, FetchRequest.PartitionData> fetch = fetchable.get(node.id());
if (fetch == null) { if (fetch == null) {
fetch = new HashMap<TopicPartition, FetchRequest.PartitionData>(); fetch = new HashMap<TopicPartition, FetchRequest.PartitionData>();
@ -273,7 +274,7 @@ public class Fetcher<K, V> {
for (Map.Entry<Integer, Map<TopicPartition, FetchRequest.PartitionData>> entry : fetchable.entrySet()) { for (Map.Entry<Integer, Map<TopicPartition, FetchRequest.PartitionData>> entry : fetchable.entrySet()) {
int nodeId = entry.getKey(); int nodeId = entry.getKey();
final FetchRequest fetch = new FetchRequest(this.maxWaitMs, this.minBytes, entry.getValue()); final FetchRequest fetch = new FetchRequest(this.maxWaitMs, this.minBytes, entry.getValue());
RequestSend send = new RequestSend(nodeId, this.client.nextRequestHeader(ApiKeys.FETCH), fetch.toStruct()); RequestSend send = new RequestSend(Integer.toString(nodeId), this.client.nextRequestHeader(ApiKeys.FETCH), fetch.toStruct());
RequestCompletionHandler handler = new RequestCompletionHandler() { RequestCompletionHandler handler = new RequestCompletionHandler() {
public void onComplete(ClientResponse response) { public void onComplete(ClientResponse response) {
handleFetchResponse(response, fetch); handleFetchResponse(response, fetch);

View File

@ -3,9 +3,9 @@
* file distributed with this work for additional information regarding copyright ownership. The ASF licenses this file * 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 * 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 * License. You may obtain a copy of the License at
* *
* http://www.apache.org/licenses/LICENSE-2.0 * 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 * 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 * 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. * specific language governing permissions and limitations under the License.
@ -18,11 +18,11 @@ import java.util.concurrent.ExecutionException;
import java.util.concurrent.Future; import java.util.concurrent.Future;
import java.util.concurrent.TimeUnit; import java.util.concurrent.TimeUnit;
import java.util.concurrent.atomic.AtomicInteger; import java.util.concurrent.atomic.AtomicInteger;
import java.util.concurrent.atomic.AtomicReference;
import org.apache.kafka.clients.ClientUtils; import org.apache.kafka.clients.ClientUtils;
import org.apache.kafka.clients.Metadata; import org.apache.kafka.clients.Metadata;
import org.apache.kafka.clients.NetworkClient; import org.apache.kafka.clients.NetworkClient;
import org.apache.kafka.clients.producer.internals.Partitioner;
import org.apache.kafka.clients.producer.internals.RecordAccumulator; import org.apache.kafka.clients.producer.internals.RecordAccumulator;
import org.apache.kafka.clients.producer.internals.Sender; import org.apache.kafka.clients.producer.internals.Sender;
import org.apache.kafka.common.Cluster; import org.apache.kafka.common.Cluster;
@ -72,11 +72,11 @@ import org.slf4j.LoggerFactory;
* props.put("buffer.memory", 33554432); * props.put("buffer.memory", 33554432);
* props.put("key.serializer", "org.apache.kafka.common.serialization.StringSerializer"); * props.put("key.serializer", "org.apache.kafka.common.serialization.StringSerializer");
* props.put("value.serializer", "org.apache.kafka.common.serialization.StringSerializer"); * props.put("value.serializer", "org.apache.kafka.common.serialization.StringSerializer");
* *
* Producer<String, String> producer = new KafkaProducer(props); * Producer<String, String> producer = new KafkaProducer(props);
* for(int i = 0; i < 100; i++) * for(int i = 0; i < 100; i++)
* producer.send(new ProducerRecord<String, String>("my-topic", Integer.toString(i), Integer.toString(i))); * producer.send(new ProducerRecord<String, String>("my-topic", Integer.toString(i), Integer.toString(i)));
* *
* producer.close(); * producer.close();
* }</pre> * }</pre>
* <p> * <p>
@ -91,25 +91,25 @@ import org.slf4j.LoggerFactory;
* we have specified will result in blocking on the full commit of the record, the slowest but most durable setting. * we have specified will result in blocking on the full commit of the record, the slowest but most durable setting.
* <p> * <p>
* If the request fails, the producer can automatically retry, though since we have specified <code>retries</code> * If the request fails, the producer can automatically retry, though since we have specified <code>retries</code>
* as 0 it won't. Enabling retries also opens up the possibility of duplicates (see the documentation on * as 0 it won't. Enabling retries also opens up the possibility of duplicates (see the documentation on
* <a href="http://kafka.apache.org/documentation.html#semantics">message delivery semantics</a> for details). * <a href="http://kafka.apache.org/documentation.html#semantics">message delivery semantics</a> for details).
* <p> * <p>
* The producer maintains buffers of unsent records for each partition. These buffers are of a size specified by * The producer maintains buffers of unsent records for each partition. These buffers are of a size specified by
* the <code>batch.size</code> config. Making this larger can result in more batching, but requires more memory (since we will * the <code>batch.size</code> config. Making this larger can result in more batching, but requires more memory (since we will
* generally have one of these buffers for each active partition). * generally have one of these buffers for each active partition).
* <p> * <p>
* By default a buffer is available to send immediately even if there is additional unused space in the buffer. However if you * By default a buffer is available to send immediately even if there is additional unused space in the buffer. However if you
* want to reduce the number of requests you can set <code>linger.ms</code> to something greater than 0. This will * want to reduce the number of requests you can set <code>linger.ms</code> to something greater than 0. This will
* instruct the producer to wait up to that number of milliseconds before sending a request in hope that more records will * instruct the producer to wait up to that number of milliseconds before sending a request in hope that more records will
* arrive to fill up the same batch. This is analogous to Nagle's algorithm in TCP. For example, in the code snippet above, * arrive to fill up the same batch. This is analogous to Nagle's algorithm in TCP. For example, in the code snippet above,
* likely all 100 records would be sent in a single request since we set our linger time to 1 millisecond. However this setting * likely all 100 records would be sent in a single request since we set our linger time to 1 millisecond. However this setting
* would add 1 millisecond of latency to our request waiting for more records to arrive if we didn't fill up the buffer. Note that * would add 1 millisecond of latency to our request waiting for more records to arrive if we didn't fill up the buffer. Note that
* records that arrive close together in time will generally batch together even with <code>linger.ms=0</code> so under heavy load * records that arrive close together in time will generally batch together even with <code>linger.ms=0</code> so under heavy load
* batching will occur regardless of the linger configuration; however setting this to something larger than 0 can lead to fewer, more * batching will occur regardless of the linger configuration; however setting this to something larger than 0 can lead to fewer, more
* efficient requests when not under maximal load at the cost of a small amount of latency. * efficient requests when not under maximal load at the cost of a small amount of latency.
* <p> * <p>
* The <code>buffer.memory</code> controls the total amount of memory available to the producer for buffering. If records * The <code>buffer.memory</code> controls the total amount of memory available to the producer for buffering. If records
* are sent faster than they can be transmitted to the server then this buffer space will be exhausted. When the buffer space is * are sent faster than they can be transmitted to the server then this buffer space will be exhausted. When the buffer space is
* exhausted additional send calls will block. For uses where you want to avoid any blocking you can set <code>block.on.buffer.full=false</code> which * exhausted additional send calls will block. For uses where you want to avoid any blocking you can set <code>block.on.buffer.full=false</code> which
* will cause the send call to result in an exception. * will cause the send call to result in an exception.
* <p> * <p>
@ -140,7 +140,7 @@ public class KafkaProducer<K, V> implements Producer<K, V> {
/** /**
* A producer is instantiated by providing a set of key-value pairs as configuration. Valid configuration strings * A producer is instantiated by providing a set of key-value pairs as configuration. Valid configuration strings
* are documented <a href="http://kafka.apache.org/documentation.html#producerconfigs">here</a>. Values can be * are documented <a href="http://kafka.apache.org/documentation.html#newproducerconfigs">here</a>. Values can be
* either strings or Objects of the appropriate type (for example a numeric configuration would accept either the * either strings or Objects of the appropriate type (for example a numeric configuration would accept either the
* string "42" or the integer 42). * string "42" or the integer 42).
* @param configs The producer configs * @param configs The producer configs
@ -152,7 +152,7 @@ public class KafkaProducer<K, V> implements Producer<K, V> {
/** /**
* A producer is instantiated by providing a set of key-value pairs as configuration, a key and a value {@link Serializer}. * A producer is instantiated by providing a set of key-value pairs as configuration, a key and a value {@link Serializer}.
* Valid configuration strings are documented <a href="http://kafka.apache.org/documentation.html#producerconfigs">here</a>. * Valid configuration strings are documented <a href="http://kafka.apache.org/documentation.html#newproducerconfigs">here</a>.
* Values can be either strings or Objects of the appropriate type (for example a numeric configuration would accept * Values can be either strings or Objects of the appropriate type (for example a numeric configuration would accept
* either the string "42" or the integer 42). * either the string "42" or the integer 42).
* @param configs The producer configs * @param configs The producer configs
@ -168,7 +168,7 @@ public class KafkaProducer<K, V> implements Producer<K, V> {
/** /**
* A producer is instantiated by providing a set of key-value pairs as configuration. Valid configuration strings * A producer is instantiated by providing a set of key-value pairs as configuration. Valid configuration strings
* are documented <a href="http://kafka.apache.org/documentation.html#producerconfigs">here</a>. * are documented <a href="http://kafka.apache.org/documentation.html#newproducerconfigs">here</a>.
* @param properties The producer configs * @param properties The producer configs
*/ */
public KafkaProducer(Properties properties) { public KafkaProducer(Properties properties) {
@ -177,7 +177,7 @@ public class KafkaProducer<K, V> implements Producer<K, V> {
/** /**
* A producer is instantiated by providing a set of key-value pairs as configuration, a key and a value {@link Serializer}. * A producer is instantiated by providing a set of key-value pairs as configuration, a key and a value {@link Serializer}.
* Valid configuration strings are documented <a href="http://kafka.apache.org/documentation.html#producerconfigs">here</a>. * Valid configuration strings are documented <a href="http://kafka.apache.org/documentation.html#newproducerconfigs">here</a>.
* @param properties The producer configs * @param properties The producer configs
* @param keySerializer The serializer for key that implements {@link Serializer}. The configure() method won't be * @param keySerializer The serializer for key that implements {@link Serializer}. The configure() method won't be
* called in the producer when the serializer is passed in directly. * called in the producer when the serializer is passed in directly.
@ -191,81 +191,89 @@ public class KafkaProducer<K, V> implements Producer<K, V> {
@SuppressWarnings("unchecked") @SuppressWarnings("unchecked")
private KafkaProducer(ProducerConfig config, Serializer<K> keySerializer, Serializer<V> valueSerializer) { private KafkaProducer(ProducerConfig config, Serializer<K> keySerializer, Serializer<V> valueSerializer) {
log.trace("Starting the Kafka producer"); try {
this.producerConfig = config; log.trace("Starting the Kafka producer");
this.time = new SystemTime(); this.producerConfig = config;
MetricConfig metricConfig = new MetricConfig().samples(config.getInt(ProducerConfig.METRICS_NUM_SAMPLES_CONFIG)) this.time = new SystemTime();
.timeWindow(config.getLong(ProducerConfig.METRICS_SAMPLE_WINDOW_MS_CONFIG), MetricConfig metricConfig = new MetricConfig().samples(config.getInt(ProducerConfig.METRICS_NUM_SAMPLES_CONFIG))
TimeUnit.MILLISECONDS); .timeWindow(config.getLong(ProducerConfig.METRICS_SAMPLE_WINDOW_MS_CONFIG),
String clientId = config.getString(ProducerConfig.CLIENT_ID_CONFIG); TimeUnit.MILLISECONDS);
if (clientId.length() <= 0) String clientId = config.getString(ProducerConfig.CLIENT_ID_CONFIG);
clientId = "producer-" + PRODUCER_CLIENT_ID_SEQUENCE.getAndIncrement(); if (clientId.length() <= 0)
String jmxPrefix = "kafka.producer"; clientId = "producer-" + PRODUCER_CLIENT_ID_SEQUENCE.getAndIncrement();
List<MetricsReporter> reporters = config.getConfiguredInstances(ProducerConfig.METRIC_REPORTER_CLASSES_CONFIG, String jmxPrefix = "kafka.producer";
MetricsReporter.class); List<MetricsReporter> reporters = config.getConfiguredInstances(ProducerConfig.METRIC_REPORTER_CLASSES_CONFIG,
reporters.add(new JmxReporter(jmxPrefix)); MetricsReporter.class);
this.metrics = new Metrics(metricConfig, reporters, time); reporters.add(new JmxReporter(jmxPrefix));
this.partitioner = new Partitioner(); this.metrics = new Metrics(metricConfig, reporters, time);
long retryBackoffMs = config.getLong(ProducerConfig.RETRY_BACKOFF_MS_CONFIG); this.partitioner = config.getConfiguredInstance(ProducerConfig.PARTITIONER_CLASS_CONFIG, Partitioner.class);
this.metadataFetchTimeoutMs = config.getLong(ProducerConfig.METADATA_FETCH_TIMEOUT_CONFIG); long retryBackoffMs = config.getLong(ProducerConfig.RETRY_BACKOFF_MS_CONFIG);
this.metadata = new Metadata(retryBackoffMs, config.getLong(ProducerConfig.METADATA_MAX_AGE_CONFIG)); this.metadataFetchTimeoutMs = config.getLong(ProducerConfig.METADATA_FETCH_TIMEOUT_CONFIG);
this.maxRequestSize = config.getInt(ProducerConfig.MAX_REQUEST_SIZE_CONFIG); this.metadata = new Metadata(retryBackoffMs, config.getLong(ProducerConfig.METADATA_MAX_AGE_CONFIG));
this.totalMemorySize = config.getLong(ProducerConfig.BUFFER_MEMORY_CONFIG); this.maxRequestSize = config.getInt(ProducerConfig.MAX_REQUEST_SIZE_CONFIG);
this.compressionType = CompressionType.forName(config.getString(ProducerConfig.COMPRESSION_TYPE_CONFIG)); this.totalMemorySize = config.getLong(ProducerConfig.BUFFER_MEMORY_CONFIG);
Map<String, String> metricTags = new LinkedHashMap<String, String>(); this.compressionType = CompressionType.forName(config.getString(ProducerConfig.COMPRESSION_TYPE_CONFIG));
metricTags.put("client-id", clientId); Map<String, String> metricTags = new LinkedHashMap<String, String>();
this.accumulator = new RecordAccumulator(config.getInt(ProducerConfig.BATCH_SIZE_CONFIG), metricTags.put("client-id", clientId);
this.totalMemorySize, this.accumulator = new RecordAccumulator(config.getInt(ProducerConfig.BATCH_SIZE_CONFIG),
this.compressionType, this.totalMemorySize,
config.getLong(ProducerConfig.LINGER_MS_CONFIG), this.compressionType,
retryBackoffMs, config.getLong(ProducerConfig.LINGER_MS_CONFIG),
config.getBoolean(ProducerConfig.BLOCK_ON_BUFFER_FULL_CONFIG), retryBackoffMs,
metrics, config.getBoolean(ProducerConfig.BLOCK_ON_BUFFER_FULL_CONFIG),
time, metrics,
metricTags); time,
List<InetSocketAddress> addresses = ClientUtils.parseAndValidateAddresses(config.getList(ProducerConfig.BOOTSTRAP_SERVERS_CONFIG)); metricTags);
this.metadata.update(Cluster.bootstrap(addresses), time.milliseconds()); List<InetSocketAddress> addresses = ClientUtils.parseAndValidateAddresses(config.getList(ProducerConfig.BOOTSTRAP_SERVERS_CONFIG));
this.metadata.update(Cluster.bootstrap(addresses), time.milliseconds());
NetworkClient client = new NetworkClient(new Selector(this.metrics, time , "producer", metricTags), NetworkClient client = new NetworkClient(
this.metadata, new Selector(config.getLong(ProducerConfig.CONNECTIONS_MAX_IDLE_MS_CONFIG), this.metrics, time, "producer", metricTags),
clientId, this.metadata,
config.getInt(ProducerConfig.MAX_IN_FLIGHT_REQUESTS_PER_CONNECTION), clientId,
config.getLong(ProducerConfig.RECONNECT_BACKOFF_MS_CONFIG), config.getInt(ProducerConfig.MAX_IN_FLIGHT_REQUESTS_PER_CONNECTION),
config.getInt(ProducerConfig.SEND_BUFFER_CONFIG), config.getLong(ProducerConfig.RECONNECT_BACKOFF_MS_CONFIG),
config.getInt(ProducerConfig.RECEIVE_BUFFER_CONFIG)); config.getInt(ProducerConfig.SEND_BUFFER_CONFIG),
this.sender = new Sender(client, config.getInt(ProducerConfig.RECEIVE_BUFFER_CONFIG));
this.metadata, this.sender = new Sender(client,
this.accumulator, this.metadata,
config.getInt(ProducerConfig.MAX_REQUEST_SIZE_CONFIG), this.accumulator,
(short) parseAcks(config.getString(ProducerConfig.ACKS_CONFIG)), config.getInt(ProducerConfig.MAX_REQUEST_SIZE_CONFIG),
config.getInt(ProducerConfig.RETRIES_CONFIG), (short) parseAcks(config.getString(ProducerConfig.ACKS_CONFIG)),
config.getInt(ProducerConfig.TIMEOUT_CONFIG), config.getInt(ProducerConfig.RETRIES_CONFIG),
this.metrics, config.getInt(ProducerConfig.TIMEOUT_CONFIG),
new SystemTime(), this.metrics,
clientId); new SystemTime(),
String ioThreadName = "kafka-producer-network-thread" + (clientId.length() > 0 ? " | " + clientId : ""); clientId);
this.ioThread = new KafkaThread(ioThreadName, this.sender, true); String ioThreadName = "kafka-producer-network-thread" + (clientId.length() > 0 ? " | " + clientId : "");
this.ioThread.start(); this.ioThread = new KafkaThread(ioThreadName, this.sender, true);
this.ioThread.start();
this.errors = this.metrics.sensor("errors"); this.errors = this.metrics.sensor("errors");
if (keySerializer == null) { if (keySerializer == null) {
this.keySerializer = config.getConfiguredInstance(ProducerConfig.KEY_SERIALIZER_CLASS_CONFIG, this.keySerializer = config.getConfiguredInstance(ProducerConfig.KEY_SERIALIZER_CLASS_CONFIG,
Serializer.class); Serializer.class);
this.keySerializer.configure(config.originals(), true); this.keySerializer.configure(config.originals(), true);
} else { } else {
this.keySerializer = keySerializer; this.keySerializer = keySerializer;
}
if (valueSerializer == null) {
this.valueSerializer = config.getConfiguredInstance(ProducerConfig.VALUE_SERIALIZER_CLASS_CONFIG,
Serializer.class);
this.valueSerializer.configure(config.originals(), false);
} else {
this.valueSerializer = valueSerializer;
}
config.logUnused();
log.debug("Kafka producer started");
} catch (Throwable t) {
// call close methods if internal objects are already constructed
// this is to prevent resource leak. see KAFKA-2121
close(0, TimeUnit.MILLISECONDS, true);
// now propagate the exception
throw new KafkaException("Failed to construct kafka producer", t);
} }
if (valueSerializer == null) {
this.valueSerializer = config.getConfiguredInstance(ProducerConfig.VALUE_SERIALIZER_CLASS_CONFIG,
Serializer.class);
this.valueSerializer.configure(config.originals(), false);
} else {
this.valueSerializer = valueSerializer;
}
config.logUnused();
log.debug("Kafka producer started");
} }
private static int parseAcks(String acksString) { private static int parseAcks(String acksString) {
@ -277,7 +285,7 @@ public class KafkaProducer<K, V> implements Producer<K, V> {
} }
/** /**
* Asynchronously send a record to a topic. Equivalent to <code>send(record, null)</code>. * Asynchronously send a record to a topic. Equivalent to <code>send(record, null)</code>.
* See {@link #send(ProducerRecord, Callback)} for details. * See {@link #send(ProducerRecord, Callback)} for details.
*/ */
@Override @Override
@ -301,7 +309,7 @@ public class KafkaProducer<K, V> implements Producer<K, V> {
* or throw any exception that occurred while sending the record. * or throw any exception that occurred while sending the record.
* <p> * <p>
* If you want to simulate a simple blocking call you can call the <code>get()</code> method immediately: * If you want to simulate a simple blocking call you can call the <code>get()</code> method immediately:
* *
* <pre> * <pre>
* {@code * {@code
* byte[] key = "key".getBytes(); * byte[] key = "key".getBytes();
@ -312,7 +320,7 @@ public class KafkaProducer<K, V> implements Producer<K, V> {
* <p> * <p>
* Fully non-blocking usage can make use of the {@link Callback} parameter to provide a callback that * Fully non-blocking usage can make use of the {@link Callback} parameter to provide a callback that
* will be invoked when the request is complete. * will be invoked when the request is complete.
* *
* <pre> * <pre>
* {@code * {@code
* ProducerRecord<byte[],byte[]> record = new ProducerRecord<byte[],byte[]>("the-topic", key, value); * ProducerRecord<byte[],byte[]> record = new ProducerRecord<byte[],byte[]>("the-topic", key, value);
@ -326,10 +334,10 @@ public class KafkaProducer<K, V> implements Producer<K, V> {
* }); * });
* } * }
* </pre> * </pre>
* *
* Callbacks for records being sent to the same partition are guaranteed to execute in order. That is, in the * Callbacks for records being sent to the same partition are guaranteed to execute in order. That is, in the
* following example <code>callback1</code> is guaranteed to execute before <code>callback2</code>: * following example <code>callback1</code> is guaranteed to execute before <code>callback2</code>:
* *
* <pre> * <pre>
* {@code * {@code
* producer.send(new ProducerRecord<byte[],byte[]>(topic, partition, key1, value1), callback1); * producer.send(new ProducerRecord<byte[],byte[]>(topic, partition, key1, value1), callback1);
@ -341,15 +349,15 @@ public class KafkaProducer<K, V> implements Producer<K, V> {
* they will delay the sending of messages from other threads. If you want to execute blocking or computationally * they will delay the sending of messages from other threads. If you want to execute blocking or computationally
* expensive callbacks it is recommended to use your own {@link java.util.concurrent.Executor} in the callback body * expensive callbacks it is recommended to use your own {@link java.util.concurrent.Executor} in the callback body
* to parallelize processing. * to parallelize processing.
* *
* @param record The record to send * @param record The record to send
* @param callback A user-supplied callback to execute when the record has been acknowledged by the server (null * @param callback A user-supplied callback to execute when the record has been acknowledged by the server (null
* indicates no callback) * indicates no callback)
* *
* @throws InterruptException If the thread is interrupted while blocked * @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 SerializationException If the key or value are not valid objects given the configured serializers
* @throws BufferExhaustedException If <code>block.on.buffer.full=false</code> and the buffer is full. * @throws BufferExhaustedException If <code>block.on.buffer.full=false</code> and the buffer is full.
* *
*/ */
@Override @Override
public Future<RecordMetadata> send(ProducerRecord<K, V> record, Callback callback) { public Future<RecordMetadata> send(ProducerRecord<K, V> record, Callback callback) {
@ -372,7 +380,7 @@ public class KafkaProducer<K, V> implements Producer<K, V> {
" to class " + producerConfig.getClass(ProducerConfig.VALUE_SERIALIZER_CLASS_CONFIG).getName() + " to class " + producerConfig.getClass(ProducerConfig.VALUE_SERIALIZER_CLASS_CONFIG).getName() +
" specified in value.serializer"); " specified in value.serializer");
} }
int partition = partitioner.partition(record.topic(), serializedKey, record.partition(), metadata.fetch()); int partition = partition(record, serializedKey, serializedValue, metadata.fetch());
int serializedSize = Records.LOG_OVERHEAD + Record.recordSize(serializedKey, serializedValue); int serializedSize = Records.LOG_OVERHEAD + Record.recordSize(serializedKey, serializedValue);
ensureValidRecordSize(serializedSize); ensureValidRecordSize(serializedSize);
TopicPartition tp = new TopicPartition(record.topic(), partition); TopicPartition tp = new TopicPartition(record.topic(), partition);
@ -444,12 +452,12 @@ public class KafkaProducer<K, V> implements Producer<K, V> {
ProducerConfig.BUFFER_MEMORY_CONFIG + ProducerConfig.BUFFER_MEMORY_CONFIG +
" configuration."); " configuration.");
} }
/** /**
* Invoking this method makes all buffered records immediately available to send (even if <code>linger.ms</code> is * Invoking this method makes all buffered records immediately available to send (even if <code>linger.ms</code> is
* greater than 0) and blocks on the completion of the requests associated with these records. The post-condition * greater than 0) and blocks on the completion of the requests associated with these records. The post-condition
* of <code>flush()</code> is that any previously sent record will have completed (e.g. <code>Future.isDone() == true</code>). * of <code>flush()</code> is that any previously sent record will have completed (e.g. <code>Future.isDone() == true</code>).
* A request is considered completed when it is successfully acknowledged * A request is considered completed when it is successfully acknowledged
* according to the <code>acks</code> configuration you have specified or else it results in an error. * according to the <code>acks</code> configuration you have specified or else it results in an error.
* <p> * <p>
* Other threads can continue sending records while one thread is blocked waiting for a flush call to complete, * Other threads can continue sending records while one thread is blocked waiting for a flush call to complete,
@ -467,10 +475,10 @@ public class KafkaProducer<K, V> implements Producer<K, V> {
* consumer.commit(); * consumer.commit();
* } * }
* </pre> * </pre>
* *
* Note that the above example may drop records if the produce request fails. If we want to ensure that this does not occur * Note that the above example may drop records if the produce request fails. If we want to ensure that this does not occur
* we need to set <code>retries=&lt;large_number&gt;</code> in our config. * we need to set <code>retries=&lt;large_number&gt;</code> in our config.
* *
* @throws InterruptException If the thread is interrupted while blocked * @throws InterruptException If the thread is interrupted while blocked
*/ */
@Override @Override
@ -509,21 +517,108 @@ public class KafkaProducer<K, V> implements Producer<K, V> {
/** /**
* Close this producer. This method blocks until all previously sent requests complete. * Close this producer. This method blocks until all previously sent requests complete.
* This method is equivalent to <code>close(Long.MAX_VALUE, TimeUnit.MILLISECONDS)</code>.
* <p>
* <strong>If close() is called from {@link Callback}, a warning message will be logged and close(0, TimeUnit.MILLISECONDS)
* will be called instead. We do this because the sender thread would otherwise try to join itself and
* block forever.</strong>
* <p/>
* @throws InterruptException If the thread is interrupted while blocked * @throws InterruptException If the thread is interrupted while blocked
*/ */
@Override @Override
public void close() { public void close() {
log.trace("Closing the Kafka producer."); close(Long.MAX_VALUE, TimeUnit.MILLISECONDS);
this.sender.initiateClose(); }
try {
this.ioThread.join(); /**
} catch (InterruptedException e) { * This method waits up to <code>timeout</code> for the producer to complete the sending of all incomplete requests.
throw new InterruptException(e); * <p>
* If the producer is unable to complete all requests before the timeout expires, this method will fail
* any unsent and unacknowledged records immediately.
* <p>
* If invoked from within a {@link Callback} this method will not block and will be equivalent to
* <code>close(0, TimeUnit.MILLISECONDS)</code>. This is done since no further sending will happen while
* blocking the I/O thread of the producer.
*
* @param timeout The maximum time to wait for producer to complete any pending requests. The value should be
* non-negative. Specifying a timeout of zero means do not wait for pending send requests to complete.
* @param timeUnit The time unit for the <code>timeout</code>
* @throws InterruptException If the thread is interrupted while blocked
* @throws IllegalArgumentException If the <code>timeout</code> is negative.
*/
@Override
public void close(long timeout, TimeUnit timeUnit) {
close(timeout, timeUnit, false);
}
private void close(long timeout, TimeUnit timeUnit, boolean swallowException) {
if (timeout < 0)
throw new IllegalArgumentException("The timeout cannot be negative.");
log.info("Closing the Kafka producer with timeoutMillis = {} ms.", timeUnit.toMillis(timeout));
// this will keep track of the first encountered exception
AtomicReference<Throwable> firstException = new AtomicReference<Throwable>();
boolean invokedFromCallback = Thread.currentThread() == this.ioThread;
if (timeout > 0) {
if (invokedFromCallback) {
log.warn("Overriding close timeout {} ms to 0 ms in order to prevent useless blocking due to self-join. " +
"This means you have incorrectly invoked close with a non-zero timeout from the producer call-back.", timeout);
} else {
// Try to close gracefully.
if (this.sender != null)
this.sender.initiateClose();
if (this.ioThread != null) {
try {
this.ioThread.join(timeUnit.toMillis(timeout));
} catch (InterruptedException t) {
firstException.compareAndSet(null, t);
log.error("Interrupted while joining ioThread", t);
}
}
}
} }
this.metrics.close();
this.keySerializer.close(); if (this.sender != null && this.ioThread != null && this.ioThread.isAlive()) {
this.valueSerializer.close(); log.info("Proceeding to force close the producer since pending requests could not be completed " +
"within timeout {} ms.", timeout);
this.sender.forceClose();
// Only join the sender thread when not calling from callback.
if (!invokedFromCallback) {
try {
this.ioThread.join();
} catch (InterruptedException e) {
firstException.compareAndSet(null, e);
}
}
}
ClientUtils.closeQuietly(metrics, "producer metrics", firstException);
ClientUtils.closeQuietly(keySerializer, "producer keySerializer", firstException);
ClientUtils.closeQuietly(valueSerializer, "producer valueSerializer", firstException);
log.debug("The Kafka producer has closed."); log.debug("The Kafka producer has closed.");
if (firstException.get() != null && !swallowException)
throw new KafkaException("Failed to close kafka producer", firstException.get());
}
/**
* computes partition for given record.
* if the record has partition returns the value otherwise
* calls configured partitioner class to compute the partition.
*/
private int partition(ProducerRecord<K, V> record, byte[] serializedKey , byte[] serializedValue, Cluster cluster) {
Integer partition = record.partition();
if (partition != null) {
List<PartitionInfo> partitions = cluster.partitionsForTopic(record.topic());
int numPartitions = partitions.size();
// they have given us a partition, use it
if (partition < 0 || partition >= numPartitions)
throw new IllegalArgumentException("Invalid partition given with record: " + partition
+ " is not in the range [0..."
+ numPartitions
+ "].");
return partition;
}
return this.partitioner.partition(record.topic(), record.key(), serializedKey, record.value(), serializedValue, cluster);
} }
private static class FutureFailure implements Future<RecordMetadata> { private static class FutureFailure implements Future<RecordMetadata> {

View File

@ -24,11 +24,13 @@ import java.util.HashMap;
import java.util.List; import java.util.List;
import java.util.Map; import java.util.Map;
import java.util.concurrent.Future; import java.util.concurrent.Future;
import java.util.concurrent.TimeUnit;
import org.apache.kafka.clients.producer.internals.FutureRecordMetadata; import org.apache.kafka.clients.producer.internals.FutureRecordMetadata;
import org.apache.kafka.clients.producer.internals.Partitioner; import org.apache.kafka.clients.producer.internals.DefaultPartitioner;
import org.apache.kafka.clients.producer.internals.ProduceRequestResult; import org.apache.kafka.clients.producer.internals.ProduceRequestResult;
import org.apache.kafka.common.*; import org.apache.kafka.common.*;
import org.apache.kafka.common.serialization.Serializer;
/** /**
@ -37,48 +39,56 @@ import org.apache.kafka.common.*;
* By default this mock will synchronously complete each send call successfully. However it can be configured to allow * By default this mock will synchronously complete each send call successfully. However it can be configured to allow
* the user to control the completion of the call and supply an optional error for the producer to throw. * the user to control the completion of the call and supply an optional error for the producer to throw.
*/ */
public class MockProducer implements Producer<byte[], byte[]> { public class MockProducer<K, V> implements Producer<K, V> {
private final Cluster cluster; private final Cluster cluster;
private final Partitioner partitioner = new Partitioner(); private final Partitioner partitioner;
private final List<ProducerRecord<byte[], byte[]>> sent; private final List<ProducerRecord<K, V>> sent;
private final Deque<Completion> completions; private final Deque<Completion> completions;
private boolean autoComplete; private boolean autoComplete;
private Map<TopicPartition, Long> offsets; private Map<TopicPartition, Long> offsets;
private final Serializer<K> keySerializer;
private final Serializer<V> valueSerializer;
/** /**
* Create a mock producer * Create a mock producer
* *
* @param cluster The cluster holding metadata for this producer * @param cluster The cluster holding metadata for this producer
* @param autoComplete If true automatically complete all requests successfully and execute the callback. Otherwise * @param autoComplete If true automatically complete all requests successfully and execute the callback. Otherwise
* the user must call {@link #completeNext()} or {@link #errorNext(RuntimeException)} after * the user must call {@link #completeNext()} or {@link #errorNext(RuntimeException)} after
* {@link #send(ProducerRecord) send()} to complete the call and unblock the @{link * {@link #send(ProducerRecord) send()} to complete the call and unblock the @{link
* java.util.concurrent.Future Future&lt;RecordMetadata&gt;} that is returned. * java.util.concurrent.Future Future&lt;RecordMetadata&gt;} that is returned.
* @param partitioner The partition strategy
* @param keySerializer The serializer for key that implements {@link Serializer}.
* @param valueSerializer The serializer for value that implements {@link Serializer}.
*/ */
public MockProducer(Cluster cluster, boolean autoComplete) { public MockProducer(Cluster cluster, boolean autoComplete, Partitioner partitioner, Serializer<K> keySerializer, Serializer<V> valueSerializer) {
this.cluster = cluster; this.cluster = cluster;
this.autoComplete = autoComplete; this.autoComplete = autoComplete;
this.partitioner = partitioner;
this.keySerializer = keySerializer;
this.valueSerializer = valueSerializer;
this.offsets = new HashMap<TopicPartition, Long>(); this.offsets = new HashMap<TopicPartition, Long>();
this.sent = new ArrayList<ProducerRecord<byte[], byte[]>>(); this.sent = new ArrayList<ProducerRecord<K, V>>();
this.completions = new ArrayDeque<Completion>(); this.completions = new ArrayDeque<Completion>();
} }
/** /**
* Create a new mock producer with invented metadata the given autoComplete setting. * Create a new mock producer with invented metadata the given autoComplete setting and key\value serializers
* *
* Equivalent to {@link #MockProducer(Cluster, boolean) new MockProducer(null, autoComplete)} * Equivalent to {@link #MockProducer(Cluster, boolean, Partitioner, Serializer, Serializer)} new MockProducer(Cluster.empty(), autoComplete, new DefaultPartitioner(), keySerializer, valueSerializer)}
*/ */
public MockProducer(boolean autoComplete) { public MockProducer(boolean autoComplete, Serializer<K> keySerializer, Serializer<V> valueSerializer) {
this(Cluster.empty(), autoComplete); this(Cluster.empty(), autoComplete, new DefaultPartitioner(), keySerializer, valueSerializer);
} }
/** /**
* Create a new auto completing mock producer * Create a new mock producer with invented metadata the given autoComplete setting, partitioner and key\value serializers
* *
* Equivalent to {@link #MockProducer(boolean) new MockProducer(true)} * Equivalent to {@link #MockProducer(Cluster, boolean, Partitioner, Serializer, Serializer)} new MockProducer(Cluster.empty(), autoComplete, partitioner, keySerializer, valueSerializer)}
*/ */
public MockProducer() { public MockProducer(boolean autoComplete, Partitioner partitioner, Serializer<K> keySerializer, Serializer<V> valueSerializer) {
this(true); this(Cluster.empty(), autoComplete, partitioner, keySerializer, valueSerializer);
} }
/** /**
@ -87,20 +97,20 @@ public class MockProducer implements Producer<byte[], byte[]> {
* @see #history() * @see #history()
*/ */
@Override @Override
public synchronized Future<RecordMetadata> send(ProducerRecord<byte[], byte[]> record) { public synchronized Future<RecordMetadata> send(ProducerRecord<K, V> record) {
return send(record, null); return send(record, null);
} }
/** /**
* Adds the record to the list of sent records. * Adds the record to the list of sent records.
* *
* @see #history() * @see #history()
*/ */
@Override @Override
public synchronized Future<RecordMetadata> send(ProducerRecord<byte[], byte[]> record, Callback callback) { public synchronized Future<RecordMetadata> send(ProducerRecord<K, V> record, Callback callback) {
int partition = 0; int partition = 0;
if (this.cluster.partitionsForTopic(record.topic()) != null) if (this.cluster.partitionsForTopic(record.topic()) != null)
partition = partitioner.partition(record.topic(), record.key(), record.partition(), this.cluster); partition = partition(record, this.cluster);
ProduceRequestResult result = new ProduceRequestResult(); ProduceRequestResult result = new ProduceRequestResult();
FutureRecordMetadata future = new FutureRecordMetadata(result, 0); FutureRecordMetadata future = new FutureRecordMetadata(result, 0);
TopicPartition topicPartition = new TopicPartition(record.topic(), partition); TopicPartition topicPartition = new TopicPartition(record.topic(), partition);
@ -128,7 +138,7 @@ public class MockProducer implements Producer<byte[], byte[]> {
return offset; return offset;
} }
} }
public synchronized void flush() { public synchronized void flush() {
while (!this.completions.isEmpty()) while (!this.completions.isEmpty())
completeNext(); completeNext();
@ -146,11 +156,15 @@ public class MockProducer implements Producer<byte[], byte[]> {
public void close() { public void close() {
} }
@Override
public void close(long timeout, TimeUnit timeUnit) {
}
/** /**
* Get the list of sent records since the last call to {@link #clear()} * Get the list of sent records since the last call to {@link #clear()}
*/ */
public synchronized List<ProducerRecord<byte[], byte[]>> history() { public synchronized List<ProducerRecord<K, V>> history() {
return new ArrayList<ProducerRecord<byte[], byte[]>>(this.sent); return new ArrayList<ProducerRecord<K, V>>(this.sent);
} }
/** /**
@ -163,7 +177,7 @@ public class MockProducer implements Producer<byte[], byte[]> {
/** /**
* Complete the earliest uncompleted call successfully. * Complete the earliest uncompleted call successfully.
* *
* @return true if there was an uncompleted call to complete * @return true if there was an uncompleted call to complete
*/ */
public synchronized boolean completeNext() { public synchronized boolean completeNext() {
@ -172,7 +186,7 @@ public class MockProducer implements Producer<byte[], byte[]> {
/** /**
* Complete the earliest uncompleted call with the given error. * Complete the earliest uncompleted call with the given error.
* *
* @return true if there was an uncompleted call to complete * @return true if there was an uncompleted call to complete
*/ */
public synchronized boolean errorNext(RuntimeException e) { public synchronized boolean errorNext(RuntimeException e) {
@ -185,6 +199,28 @@ public class MockProducer implements Producer<byte[], byte[]> {
} }
} }
/**
* computes partition for given record.
*/
private int partition(ProducerRecord<K, V> record, Cluster cluster) {
Integer partition = record.partition();
String topic = record.topic();
if (partition != null) {
List<PartitionInfo> partitions = cluster.partitionsForTopic(topic);
int numPartitions = partitions.size();
// they have given us a partition, use it
if (partition < 0 || partition >= numPartitions)
throw new IllegalArgumentException("Invalid partition given with record: " + partition
+ " is not in the range [0..."
+ numPartitions
+ "].");
return partition;
}
byte[] keyBytes = keySerializer.serialize(topic, record.key());
byte[] valueBytes = valueSerializer.serialize(topic, record.value());
return this.partitioner.partition(topic, record.key(), keyBytes, record.value(), valueBytes, cluster);
}
private static class Completion { private static class Completion {
private final long offset; private final long offset;
private final RecordMetadata metadata; private final RecordMetadata metadata;

View File

@ -0,0 +1,46 @@
/**
* Licensed to the Apache Software Foundation (ASF) under one or more
* contributor license agreements. See the NOTICE file distributed with
* this work for additional information regarding copyright ownership.
* The ASF licenses this file to You under the Apache License, Version 2.0
* (the "License"); you may not use this file except in compliance with
* the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.kafka.clients.producer;
import org.apache.kafka.common.Configurable;
import org.apache.kafka.common.Cluster;
/**
* Partitioner Interface
*/
public interface Partitioner extends Configurable {
/**
* Compute the partition for the given record.
*
* @param topic The topic name
* @param key The key to partition on (or null if no key)
* @param keyBytes The serialized key to partition on( or null if no key)
* @param value The value to partition on or null
* @param valueBytes The serialized value to partition on or null
* @param cluster The current cluster metadata
*/
public int partition(String topic, Object key, byte[] keyBytes, Object value, byte[] valueBytes, Cluster cluster);
/**
* This is called when partitioner is closed.
*/
public void close();
}

View File

@ -20,6 +20,7 @@ import java.io.Closeable;
import java.util.List; import java.util.List;
import java.util.Map; import java.util.Map;
import java.util.concurrent.Future; import java.util.concurrent.Future;
import java.util.concurrent.TimeUnit;
import org.apache.kafka.common.Metric; import org.apache.kafka.common.Metric;
import org.apache.kafka.common.PartitionInfo; import org.apache.kafka.common.PartitionInfo;
@ -67,4 +68,10 @@ public interface Producer<K, V> extends Closeable {
*/ */
public void close(); public void close();
/**
* Tries to close the producer cleanly within the specified timeout. If the close does not complete within the
* timeout, fail any pending send requests and force close the producer.
*/
public void close(long timeout, TimeUnit unit);
} }

View File

@ -3,9 +3,9 @@
* file distributed with this work for additional information regarding copyright ownership. The ASF licenses this file * 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 * 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 * License. You may obtain a copy of the License at
* *
* http://www.apache.org/licenses/LICENSE-2.0 * 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 * 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 * 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. * specific language governing permissions and limitations under the License.
@ -29,7 +29,7 @@ import org.apache.kafka.common.serialization.Serializer;
/** /**
* Configuration for the Kafka Producer. Documentation for these configurations can be found in the <a * Configuration for the Kafka Producer. Documentation for these configurations can be found in the <a
* href="http://kafka.apache.org/documentation.html#new-producer">Kafka documentation</a> * href="http://kafka.apache.org/documentation.html#newproducerconfigs">Kafka documentation</a>
*/ */
public class ProducerConfig extends AbstractConfig { public class ProducerConfig extends AbstractConfig {
@ -51,7 +51,7 @@ public class ProducerConfig extends AbstractConfig {
/** <code>metadata.max.age.ms</code> */ /** <code>metadata.max.age.ms</code> */
public static final String METADATA_MAX_AGE_CONFIG = CommonClientConfigs.METADATA_MAX_AGE_CONFIG; public static final String METADATA_MAX_AGE_CONFIG = CommonClientConfigs.METADATA_MAX_AGE_CONFIG;
private static final String METADATA_MAX_AGE_DOC = CommonClientConfigs.METADATA_MAX_AGE_DOC; private static final String METADATA_MAX_AGE_DOC = CommonClientConfigs.METADATA_MAX_AGE_DOC;
/** <code>batch.size</code> */ /** <code>batch.size</code> */
public static final String BATCH_SIZE_CONFIG = "batch.size"; public static final String BATCH_SIZE_CONFIG = "batch.size";
private static final String BATCH_SIZE_DOC = "The producer will attempt to batch records together into fewer requests whenever multiple records are being sent" + " to the same partition. This helps performance on both the client and the server. This configuration controls the " private static final String BATCH_SIZE_DOC = "The producer will attempt to batch records together into fewer requests whenever multiple records are being sent" + " to the same partition. This helps performance on both the client and the server. This configuration controls the "
@ -169,6 +169,13 @@ public class ProducerConfig extends AbstractConfig {
public static final String VALUE_SERIALIZER_CLASS_CONFIG = "value.serializer"; public static final String VALUE_SERIALIZER_CLASS_CONFIG = "value.serializer";
private static final String VALUE_SERIALIZER_CLASS_DOC = "Serializer class for value that implements the <code>Serializer</code> interface."; private static final String VALUE_SERIALIZER_CLASS_DOC = "Serializer class for value that implements the <code>Serializer</code> interface.";
/** <code>connections.max.idle.ms</code> */
public static final String CONNECTIONS_MAX_IDLE_MS_CONFIG = CommonClientConfigs.CONNECTIONS_MAX_IDLE_MS_CONFIG;
/** <code>partitioner.class</code> */
public static final String PARTITIONER_CLASS_CONFIG = "partitioner.class";
private static final String PARTITIONER_CLASS_DOC = "Partitioner class that implements the <code>Partitioner</code> interface.";
static { static {
CONFIG = new ConfigDef().define(BOOTSTRAP_SERVERS_CONFIG, Type.LIST, Importance.HIGH, CommonClientConfigs.BOOSTRAP_SERVERS_DOC) CONFIG = new ConfigDef().define(BOOTSTRAP_SERVERS_CONFIG, Type.LIST, Importance.HIGH, CommonClientConfigs.BOOSTRAP_SERVERS_DOC)
.define(BUFFER_MEMORY_CONFIG, Type.LONG, 32 * 1024 * 1024L, atLeast(0L), Importance.HIGH, BUFFER_MEMORY_DOC) .define(BUFFER_MEMORY_CONFIG, Type.LONG, 32 * 1024 * 1024L, atLeast(0L), Importance.HIGH, BUFFER_MEMORY_DOC)
@ -217,7 +224,10 @@ public class ProducerConfig extends AbstractConfig {
Importance.LOW, Importance.LOW,
MAX_IN_FLIGHT_REQUESTS_PER_CONNECTION_DOC) MAX_IN_FLIGHT_REQUESTS_PER_CONNECTION_DOC)
.define(KEY_SERIALIZER_CLASS_CONFIG, Type.CLASS, Importance.HIGH, KEY_SERIALIZER_CLASS_DOC) .define(KEY_SERIALIZER_CLASS_CONFIG, Type.CLASS, Importance.HIGH, KEY_SERIALIZER_CLASS_DOC)
.define(VALUE_SERIALIZER_CLASS_CONFIG, Type.CLASS, Importance.HIGH, VALUE_SERIALIZER_CLASS_DOC); .define(VALUE_SERIALIZER_CLASS_CONFIG, Type.CLASS, Importance.HIGH, VALUE_SERIALIZER_CLASS_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, 9 * 60 * 1000, Importance.MEDIUM, CommonClientConfigs.CONNECTIONS_MAX_IDLE_MS_DOC)
.define(PARTITIONER_CLASS_CONFIG, Type.CLASS, "org.apache.kafka.clients.producer.internals.DefaultPartitioner", Importance.MEDIUM, PARTITIONER_CLASS_DOC);
} }
public static Map<String, Object> addSerializerToConfig(Map<String, Object> configs, public static Map<String, Object> addSerializerToConfig(Map<String, Object> configs,
@ -242,7 +252,7 @@ public class ProducerConfig extends AbstractConfig {
return newProperties; return newProperties;
} }
ProducerConfig(Map<? extends Object, ? extends Object> props) { ProducerConfig(Map<?, ?> props) {
super(CONFIG, props); super(CONFIG, props);
} }

View File

@ -17,9 +17,11 @@
package org.apache.kafka.clients.producer.internals; package org.apache.kafka.clients.producer.internals;
import java.util.List; import java.util.List;
import java.util.Map;
import java.util.Random; import java.util.Random;
import java.util.concurrent.atomic.AtomicInteger; import java.util.concurrent.atomic.AtomicInteger;
import org.apache.kafka.clients.producer.Partitioner;
import org.apache.kafka.common.Cluster; import org.apache.kafka.common.Cluster;
import org.apache.kafka.common.PartitionInfo; import org.apache.kafka.common.PartitionInfo;
import org.apache.kafka.common.utils.Utils; import org.apache.kafka.common.utils.Utils;
@ -31,59 +33,57 @@ import org.apache.kafka.common.utils.Utils;
* <li>If no partition is specified but a key is present choose a partition based on a hash of the key * <li>If no partition is specified but a key is present choose a partition based on a hash of the key
* <li>If no partition or key is present choose a partition in a round-robin fashion * <li>If no partition or key is present choose a partition in a round-robin fashion
*/ */
public class Partitioner { public class DefaultPartitioner implements Partitioner {
private final AtomicInteger counter = new AtomicInteger(new Random().nextInt()); private final AtomicInteger counter = new AtomicInteger(new Random().nextInt());
/** /**
* A cheap way to deterministically convert a number to a positive value. When the input is * A cheap way to deterministically convert a number to a positive value. When the input is
* positive, the original value is returned. When the input number is negative, the returned * positive, the original value is returned. When the input number is negative, the returned
* positive value is the original value bit AND against 0x7fffffff which is not its absolutely * positive value is the original value bit AND against 0x7fffffff which is not its absolutely
* value. * value.
* *
* Note: changing this method in the future will possibly cause partition selection not to be * Note: changing this method in the future will possibly cause partition selection not to be
* compatible with the existing messages already placed on a partition. * compatible with the existing messages already placed on a partition.
* *
* @param number a given number * @param number a given number
* @return a positive number. * @return a positive number.
*/ */
private static int toPositive(int number) { private static int toPositive(int number) {
return number & 0x7fffffff; return number & 0x7fffffff;
} }
public void configure(Map<String, ?> configs) {}
/** /**
* Compute the partition for the given record. * Compute the partition for the given record.
* *
* @param topic The topic name * @param topic The topic name
* @param key The key to partition on (or null if no key) * @param key The key to partition on (or null if no key)
* @param partition The partition to use (or null if none) * @param keyBytes serialized key to partition on (or null if no key)
* @param value The value to partition on or null
* @param valueBytes serialized value to partition on or null
* @param cluster The current cluster metadata * @param cluster The current cluster metadata
*/ */
public int partition(String topic, byte[] key, Integer partition, Cluster cluster) { public int partition(String topic, Object key, byte[] keyBytes, Object value, byte[] valueBytes, Cluster cluster) {
List<PartitionInfo> partitions = cluster.partitionsForTopic(topic); List<PartitionInfo> partitions = cluster.partitionsForTopic(topic);
int numPartitions = partitions.size(); int numPartitions = partitions.size();
if (partition != null) { if (keyBytes == null) {
// they have given us a partition, use it
if (partition < 0 || partition >= numPartitions)
throw new IllegalArgumentException("Invalid partition given with record: " + partition
+ " is not in the range [0..."
+ numPartitions
+ "].");
return partition;
} else if (key == null) {
int nextValue = counter.getAndIncrement(); int nextValue = counter.getAndIncrement();
List<PartitionInfo> availablePartitions = cluster.availablePartitionsForTopic(topic); List<PartitionInfo> availablePartitions = cluster.availablePartitionsForTopic(topic);
if (availablePartitions.size() > 0) { if (availablePartitions.size() > 0) {
int part = Partitioner.toPositive(nextValue) % availablePartitions.size(); int part = DefaultPartitioner.toPositive(nextValue) % availablePartitions.size();
return availablePartitions.get(part).partition(); return availablePartitions.get(part).partition();
} else { } else {
// no partitions are available, give a non-available partition // no partitions are available, give a non-available partition
return Partitioner.toPositive(nextValue) % numPartitions; return DefaultPartitioner.toPositive(nextValue) % numPartitions;
} }
} else { } else {
// hash the key to choose a partition // hash the keyBytes to choose a partition
return Partitioner.toPositive(Utils.murmur2(key)) % numPartitions; return DefaultPartitioner.toPositive(Utils.murmur2(keyBytes)) % numPartitions;
} }
} }
public void close() {}
} }

View File

@ -56,8 +56,9 @@ public final class RecordAccumulator {
private static final Logger log = LoggerFactory.getLogger(RecordAccumulator.class); private static final Logger log = LoggerFactory.getLogger(RecordAccumulator.class);
private volatile boolean closed; private volatile boolean closed;
private volatile AtomicInteger flushesInProgress;
private int drainIndex; private int drainIndex;
private final AtomicInteger flushesInProgress;
private final AtomicInteger appendsInProgress;
private final int batchSize; private final int batchSize;
private final CompressionType compression; private final CompressionType compression;
private final long lingerMs; private final long lingerMs;
@ -67,6 +68,7 @@ public final class RecordAccumulator {
private final ConcurrentMap<TopicPartition, Deque<RecordBatch>> batches; private final ConcurrentMap<TopicPartition, Deque<RecordBatch>> batches;
private final IncompleteRecordBatches incomplete; private final IncompleteRecordBatches incomplete;
/** /**
* Create a new record accumulator * Create a new record accumulator
* *
@ -96,6 +98,7 @@ public final class RecordAccumulator {
this.drainIndex = 0; this.drainIndex = 0;
this.closed = false; this.closed = false;
this.flushesInProgress = new AtomicInteger(0); this.flushesInProgress = new AtomicInteger(0);
this.appendsInProgress = new AtomicInteger(0);
this.batchSize = batchSize; this.batchSize = batchSize;
this.compression = compression; this.compression = compression;
this.lingerMs = lingerMs; this.lingerMs = lingerMs;
@ -146,40 +149,50 @@ public final class RecordAccumulator {
* @param callback The user-supplied callback to execute when the request is complete * @param callback The user-supplied callback to execute when the request is complete
*/ */
public RecordAppendResult append(TopicPartition tp, byte[] key, byte[] value, Callback callback) throws InterruptedException { public RecordAppendResult append(TopicPartition tp, byte[] key, byte[] value, Callback callback) throws InterruptedException {
if (closed) // We keep track of the number of appending thread to make sure we do not miss batches in
throw new IllegalStateException("Cannot send after the producer is closed."); // abortIncompleteBatches().
// check if we have an in-progress batch appendsInProgress.incrementAndGet();
Deque<RecordBatch> dq = dequeFor(tp); try {
synchronized (dq) { if (closed)
RecordBatch last = dq.peekLast(); throw new IllegalStateException("Cannot send after the producer is closed.");
if (last != null) { // check if we have an in-progress batch
FutureRecordMetadata future = last.tryAppend(key, value, callback); Deque<RecordBatch> dq = dequeFor(tp);
if (future != null) synchronized (dq) {
return new RecordAppendResult(future, dq.size() > 1 || last.records.isFull(), false); RecordBatch last = dq.peekLast();
} if (last != null) {
} FutureRecordMetadata future = last.tryAppend(key, value, callback);
if (future != null)
// we don't have an in-progress record batch try to allocate a new batch return new RecordAppendResult(future, dq.size() > 1 || last.records.isFull(), false);
int size = Math.max(this.batchSize, Records.LOG_OVERHEAD + Record.recordSize(key, value));
log.trace("Allocating a new {} byte message buffer for topic {} partition {}", size, tp.topic(), tp.partition());
ByteBuffer buffer = free.allocate(size);
synchronized (dq) {
RecordBatch last = dq.peekLast();
if (last != null) {
FutureRecordMetadata future = last.tryAppend(key, value, callback);
if (future != null) {
// Somebody else found us a batch, return the one we waited for! Hopefully this doesn't happen often...
free.deallocate(buffer);
return new RecordAppendResult(future, dq.size() > 1 || last.records.isFull(), false);
} }
} }
MemoryRecords records = MemoryRecords.emptyRecords(buffer, compression, this.batchSize);
RecordBatch batch = new RecordBatch(tp, records, time.milliseconds());
FutureRecordMetadata future = Utils.notNull(batch.tryAppend(key, value, callback));
dq.addLast(batch); // we don't have an in-progress record batch try to allocate a new batch
incomplete.add(batch); int size = Math.max(this.batchSize, Records.LOG_OVERHEAD + Record.recordSize(key, value));
return new RecordAppendResult(future, dq.size() > 1 || batch.records.isFull(), true); log.trace("Allocating a new {} byte message buffer for topic {} partition {}", size, tp.topic(), tp.partition());
ByteBuffer buffer = free.allocate(size);
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.");
RecordBatch last = dq.peekLast();
if (last != null) {
FutureRecordMetadata future = last.tryAppend(key, value, callback);
if (future != null) {
// Somebody else found us a batch, return the one we waited for! Hopefully this doesn't happen often...
free.deallocate(buffer);
return new RecordAppendResult(future, dq.size() > 1 || last.records.isFull(), false);
}
}
MemoryRecords records = MemoryRecords.emptyRecords(buffer, compression, this.batchSize);
RecordBatch batch = new RecordBatch(tp, records, time.milliseconds());
FutureRecordMetadata future = Utils.notNull(batch.tryAppend(key, value, callback));
dq.addLast(batch);
incomplete.add(batch);
return new RecordAppendResult(future, dq.size() > 1 || batch.records.isFull(), true);
}
} finally {
appendsInProgress.decrementAndGet();
} }
} }
@ -292,17 +305,21 @@ public final class RecordAccumulator {
synchronized (deque) { synchronized (deque) {
RecordBatch first = deque.peekFirst(); RecordBatch first = deque.peekFirst();
if (first != null) { if (first != null) {
if (size + first.records.sizeInBytes() > maxSize && !ready.isEmpty()) { boolean backoff = first.attempts > 0 && first.lastAttemptMs + retryBackoffMs > now;
// there is a rare case that a single batch size is larger than the request size due // Only drain the batch if it is not during backoff period.
// to compression; in this case we will still eventually send this batch in a single if (!backoff) {
// request if (size + first.records.sizeInBytes() > maxSize && !ready.isEmpty()) {
break; // there is a rare case that a single batch size is larger than the request size due
} else { // to compression; in this case we will still eventually send this batch in a single
RecordBatch batch = deque.pollFirst(); // request
batch.records.close(); break;
size += batch.records.sizeInBytes(); } else {
ready.add(batch); RecordBatch batch = deque.pollFirst();
batch.drainedMs = now; batch.records.close();
size += batch.records.sizeInBytes();
ready.add(batch);
batch.drainedMs = now;
}
} }
} }
} }
@ -347,7 +364,14 @@ public final class RecordAccumulator {
public void beginFlush() { public void beginFlush() {
this.flushesInProgress.getAndIncrement(); this.flushesInProgress.getAndIncrement();
} }
/**
* Are there any threads currently appending messages?
*/
private boolean appendsInProgress() {
return appendsInProgress.get() > 0;
}
/** /**
* Mark all partitions as ready to send and block until the send is complete * Mark all partitions as ready to send and block until the send is complete
*/ */
@ -357,6 +381,40 @@ public final class RecordAccumulator {
this.flushesInProgress.decrementAndGet(); this.flushesInProgress.decrementAndGet();
} }
/**
* This function is only called when sender is closed forcefully. It will fail all the
* incomplete batches and return.
*/
public void abortIncompleteBatches() {
// We need to keep aborting the incomplete batch until no thread is trying to append to
// 1. Avoid losing batches.
// 2. Free up memory in case appending threads are blocked on buffer full.
// This is a tight loop but should be able to get through very quickly.
do {
abortBatches();
} while (appendsInProgress());
// After this point, no thread will append any messages because they will see the close
// flag set. We need to do the last abort after no thread was appending in case the there was a new
// batch appended by the last appending thread.
abortBatches();
this.batches.clear();
}
/**
* Go through incomplete batches and abort them.
*/
private void abortBatches() {
for (RecordBatch batch : incomplete.all()) {
Deque<RecordBatch> dq = dequeFor(batch.topicPartition);
// Close the batch before aborting
synchronized (dq) {
batch.records.close();
}
batch.done(-1L, new IllegalStateException("Producer is closed forcefully."));
deallocate(batch);
}
}
/** /**
* Close this accumulator and force all the record buffers to be drained * Close this accumulator and force all the record buffers to be drained
*/ */
@ -399,7 +457,7 @@ public final class RecordAccumulator {
*/ */
private final static class IncompleteRecordBatches { private final static class IncompleteRecordBatches {
private final Set<RecordBatch> incomplete; private final Set<RecordBatch> incomplete;
public IncompleteRecordBatches() { public IncompleteRecordBatches() {
this.incomplete = new HashSet<RecordBatch>(); this.incomplete = new HashSet<RecordBatch>();
} }

View File

@ -83,6 +83,9 @@ public class Sender implements Runnable {
/* true while the sender thread is still running */ /* true while the sender thread is still running */
private volatile boolean running; private volatile boolean running;
/* true when the caller wants to ignore all unsent/inflight messages and force close. */
private volatile boolean forceClose;
/* metrics */ /* metrics */
private final SenderMetrics sensors; private final SenderMetrics sensors;
@ -132,15 +135,23 @@ public class Sender implements Runnable {
// okay we stopped accepting requests but there may still be // okay we stopped accepting requests but there may still be
// requests in the accumulator or waiting for acknowledgment, // requests in the accumulator or waiting for acknowledgment,
// wait until these are completed. // wait until these are completed.
while (this.accumulator.hasUnsent() || this.client.inFlightRequestCount() > 0) { while (!forceClose && (this.accumulator.hasUnsent() || this.client.inFlightRequestCount() > 0)) {
try { try {
run(time.milliseconds()); run(time.milliseconds());
} catch (Exception e) { } catch (Exception e) {
log.error("Uncaught error in kafka producer I/O thread: ", e); log.error("Uncaught error in kafka producer I/O thread: ", e);
} }
} }
if (forceClose) {
this.client.close(); // We need to fail all the incomplete batches and wake up the threads waiting on
// the futures.
this.accumulator.abortIncompleteBatches();
}
try {
this.client.close();
} catch (Exception e) {
log.error("Failed to close network client", e);
}
log.debug("Shutdown of Kafka producer I/O thread has completed."); log.debug("Shutdown of Kafka producer I/O thread has completed.");
} }
@ -178,7 +189,6 @@ public class Sender implements Runnable {
now); now);
sensors.updateProduceRequestMetrics(batches); sensors.updateProduceRequestMetrics(batches);
List<ClientRequest> requests = createProduceRequests(batches, now); List<ClientRequest> requests = createProduceRequests(batches, now);
// If we have any nodes that are ready to send + have sendable data, poll with 0 timeout so this can immediately // If we have any nodes that are ready to send + have sendable data, poll with 0 timeout so this can immediately
// loop and try sending more data. Otherwise, the timeout is determined by nodes that have partitions with data // loop and try sending more data. Otherwise, the timeout is determined by nodes that have partitions with data
// that isn't yet sendable (e.g. lingering, backing off). Note that this specifically does not include nodes // that isn't yet sendable (e.g. lingering, backing off). Note that this specifically does not include nodes
@ -208,6 +218,14 @@ public class Sender implements Runnable {
this.wakeup(); this.wakeup();
} }
/**
* Closes the sender without sending out any pending messages.
*/
public void forceClose() {
this.forceClose = true;
initiateClose();
}
/** /**
* Handle a produce response * Handle a produce response
*/ */
@ -302,7 +320,7 @@ public class Sender implements Runnable {
recordsByPartition.put(tp, batch); recordsByPartition.put(tp, batch);
} }
ProduceRequest request = new ProduceRequest(acks, timeout, produceRecordsByPartition); ProduceRequest request = new ProduceRequest(acks, timeout, produceRecordsByPartition);
RequestSend send = new RequestSend(destination, RequestSend send = new RequestSend(Integer.toString(destination),
this.client.nextRequestHeader(ApiKeys.PRODUCE), this.client.nextRequestHeader(ApiKeys.PRODUCE),
request.toStruct()); request.toStruct());
RequestCompletionHandler callback = new RequestCompletionHandler() { RequestCompletionHandler callback = new RequestCompletionHandler() {
@ -392,7 +410,7 @@ public class Sender implements Runnable {
m = new MetricName("metadata-age", metricGrpName, "The age in seconds of the current producer metadata being used.", metricTags); m = new MetricName("metadata-age", metricGrpName, "The age in seconds of the current producer metadata being used.", metricTags);
metrics.addMetric(m, new Measurable() { metrics.addMetric(m, new Measurable() {
public double measure(MetricConfig config, long now) { public double measure(MetricConfig config, long now) {
return (now - metadata.lastUpdate()) / 1000.0; return (now - metadata.lastSuccessfulUpdate()) / 1000.0;
} }
}); });
} }
@ -487,10 +505,10 @@ public class Sender implements Runnable {
topicErrorSensor.record(count, now); topicErrorSensor.record(count, now);
} }
public void recordLatency(int node, long latency) { public void recordLatency(String node, long latency) {
long now = time.milliseconds(); long now = time.milliseconds();
this.requestTimeSensor.record(latency, now); this.requestTimeSensor.record(latency, now);
if (node >= 0) { if (!node.isEmpty()) {
String nodeTimeName = "node-" + node + ".latency"; String nodeTimeName = "node-" + node + ".latency";
Sensor nodeRequestTime = this.metrics.getSensor(nodeTimeName); Sensor nodeRequestTime = this.metrics.getSensor(nodeTimeName);
if (nodeRequestTime != null) if (nodeRequestTime != null)

View File

@ -18,12 +18,14 @@ package org.apache.kafka.common;
public class Node { public class Node {
private final int id; private final int id;
private final String idString;
private final String host; private final String host;
private final int port; private final int port;
public Node(int id, String host, int port) { public Node(int id, String host, int port) {
super(); super();
this.id = id; this.id = id;
this.idString = Integer.toString(id);
this.host = host; this.host = host;
this.port = port; this.port = port;
} }
@ -39,6 +41,14 @@ public class Node {
return id; return id;
} }
/**
* String representation of the node id.
* Typically the integer id is used to serialize over the wire, the string representation is used as an identifier with NetworkClient code
*/
public String idString() {
return idString;
}
/** /**
* The host name for this node * The host name for this node
*/ */

View File

@ -31,4 +31,9 @@ public class InterruptException extends KafkaException {
Thread.currentThread().interrupt(); Thread.currentThread().interrupt();
} }
public InterruptException(String message) {
super(message, new InterruptedException());
Thread.currentThread().interrupt();
}
} }

View File

@ -15,7 +15,7 @@ package org.apache.kafka.common.errors;
/** /**
* This topic/partition doesn't exist * This topic/partition doesn't exist
*/ */
public class UnknownTopicOrPartitionException extends RetriableException { public class UnknownTopicOrPartitionException extends InvalidMetadataException {
private static final long serialVersionUID = 1L; private static final long serialVersionUID = 1L;

View File

@ -12,6 +12,7 @@
*/ */
package org.apache.kafka.common.metrics; package org.apache.kafka.common.metrics;
import java.io.Closeable;
import java.util.ArrayList; import java.util.ArrayList;
import java.util.List; import java.util.List;
import java.util.Map; import java.util.Map;
@ -46,7 +47,7 @@ import org.apache.kafka.common.utils.Utils;
* sensor.record(messageSize); * sensor.record(messageSize);
* </pre> * </pre>
*/ */
public class Metrics { public class Metrics implements Closeable {
private final MetricConfig config; private final MetricConfig config;
private final ConcurrentMap<MetricName, KafkaMetric> metrics; private final ConcurrentMap<MetricName, KafkaMetric> metrics;
@ -192,6 +193,7 @@ public class Metrics {
/** /**
* Close this metrics repository. * Close this metrics repository.
*/ */
@Override
public void close() { public void close() {
for (MetricsReporter reporter : this.reporters) for (MetricsReporter reporter : this.reporters)
reporter.close(); reporter.close();

View File

@ -58,7 +58,7 @@ public class Histogram {
@Override @Override
public String toString() { public String toString() {
StringBuilder b = new StringBuilder('{'); StringBuilder b = new StringBuilder("{");
for (int i = 0; i < this.hist.length - 1; i++) { for (int i = 0; i < this.hist.length - 1; i++) {
b.append(String.format("%.10f", binScheme.fromBin(i))); b.append(String.format("%.10f", binScheme.fromBin(i)));
b.append(':'); b.append(':');

View File

@ -25,11 +25,11 @@ import java.nio.channels.ScatteringByteChannel;
*/ */
public class ByteBufferReceive implements Receive { public class ByteBufferReceive implements Receive {
private final int source; private final String source;
private final ByteBuffer[] buffers; private final ByteBuffer[] buffers;
private int remaining; private int remaining;
public ByteBufferReceive(int source, ByteBuffer... buffers) { public ByteBufferReceive(String source, ByteBuffer... buffers) {
super(); super();
this.source = source; this.source = source;
this.buffers = buffers; this.buffers = buffers;
@ -38,7 +38,7 @@ public class ByteBufferReceive implements Receive {
} }
@Override @Override
public int source() { public String source() {
return source; return source;
} }
@ -54,8 +54,4 @@ public class ByteBufferReceive implements Receive {
return read; return read;
} }
public ByteBuffer[] reify() {
return buffers;
}
} }

View File

@ -22,12 +22,12 @@ import java.nio.channels.GatheringByteChannel;
*/ */
public class ByteBufferSend implements Send { public class ByteBufferSend implements Send {
private final int destination; private final String destination;
protected final ByteBuffer[] buffers; protected final ByteBuffer[] buffers;
private int remaining; private int remaining;
private int size; private int size;
public ByteBufferSend(int destination, ByteBuffer... buffers) { public ByteBufferSend(String destination, ByteBuffer... buffers) {
super(); super();
this.destination = destination; this.destination = destination;
this.buffers = buffers; this.buffers = buffers;
@ -37,7 +37,7 @@ public class ByteBufferSend implements Send {
} }
@Override @Override
public int destination() { public String destination() {
return destination; return destination;
} }
@ -47,16 +47,7 @@ public class ByteBufferSend implements Send {
} }
@Override @Override
public ByteBuffer[] reify() { public long size() {
return this.buffers;
}
@Override
public int remaining() {
return this.remaining;
}
public int size() {
return this.size; return this.size;
} }
@ -64,9 +55,8 @@ public class ByteBufferSend implements Send {
public long writeTo(GatheringByteChannel channel) throws IOException { public long writeTo(GatheringByteChannel channel) throws IOException {
long written = channel.write(buffers); long written = channel.write(buffers);
if (written < 0) if (written < 0)
throw new EOFException("This shouldn't happen."); throw new EOFException("Wrote negative bytes to channel. This shouldn't happen.");
remaining -= written; remaining -= written;
return written; return written;
} }
} }

View File

@ -5,8 +5,8 @@
* The ASF licenses this file to You under the Apache License, Version 2.0 * 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 not use this file except in compliance with
* the License. You may obtain a copy of the License at * the License. You may obtain a copy of the License at
* *
* http://www.apache.org/licenses/LICENSE-2.0 * http://www.apache.org/licenses/LICENSE-2.0
* *
* Unless required by applicable law or agreed to in writing, software * Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS, * distributed under the License is distributed on an "AS IS" BASIS,
@ -14,27 +14,17 @@
* See the License for the specific language governing permissions and * See the License for the specific language governing permissions and
* limitations under the License. * limitations under the License.
*/ */
package org.apache.kafka.common.network;
package kafka.network import org.apache.kafka.common.KafkaException;
import java.nio._ public class InvalidReceiveException extends KafkaException {
import java.nio.channels._
import kafka.utils._
@nonthreadsafe public InvalidReceiveException(String message) {
private[kafka] class ByteBufferSend(val buffer: ByteBuffer) extends Send { super(message);
}
var complete: Boolean = false
def this(size: Int) = this(ByteBuffer.allocate(size)) public InvalidReceiveException(String message, Throwable cause) {
super(message, cause);
def writeTo(channel: GatheringByteChannel): Int = { }
expectIncomplete()
var written = 0
written += channel.write(buffer)
if(!buffer.hasRemaining)
complete = true
written
}
} }

View File

@ -0,0 +1,100 @@
/**
* 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.network;
import org.apache.kafka.common.KafkaException;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
import java.io.IOException;
import java.nio.channels.GatheringByteChannel;
import java.util.Iterator;
import java.util.List;
/**
* A set of composite sends, sent one after another
*/
public class MultiSend implements Send {
private static final Logger log = LoggerFactory.getLogger(MultiSend.class);
private String dest;
private long totalWritten = 0;
private List<Send> sends;
private Iterator<Send> sendsIterator;
private Send current;
private boolean doneSends = false;
private long size = 0;
public MultiSend(String dest, List<Send> sends) {
this.dest = dest;
this.sends = sends;
this.sendsIterator = sends.iterator();
nextSendOrDone();
for (Send send: sends)
this.size += send.size();
}
@Override
public long size() {
return size;
}
@Override
public String destination() {
return dest;
}
@Override
public boolean completed() {
if (doneSends) {
if (totalWritten != size)
log.error("mismatch in sending bytes over socket; expected: " + size + " actual: " + totalWritten);
return true;
} else {
return false;
}
}
@Override
public long writeTo(GatheringByteChannel channel) throws IOException {
if (completed())
throw new KafkaException("This operation cannot be completed on a complete request.");
int totalWrittenPerCall = 0;
boolean sendComplete = false;
do {
long written = current.writeTo(channel);
totalWritten += written;
totalWrittenPerCall += written;
sendComplete = current.completed();
if (sendComplete)
nextSendOrDone();
} while (!completed() && sendComplete);
if (log.isTraceEnabled())
log.trace("Bytes written as part of multisend call : " + totalWrittenPerCall + "Total bytes written so far : " + totalWritten + "Expected bytes to write : " + size);
return totalWrittenPerCall;
}
// update current if there's a next Send, mark sends as done if there isn't
private void nextSendOrDone() {
if (sendsIterator.hasNext())
current = sendsIterator.next();
else
doneSends = true;
}
}

View File

@ -15,6 +15,7 @@ package org.apache.kafka.common.network;
import java.io.EOFException; import java.io.EOFException;
import java.io.IOException; import java.io.IOException;
import java.nio.ByteBuffer; import java.nio.ByteBuffer;
import java.nio.channels.ReadableByteChannel;
import java.nio.channels.ScatteringByteChannel; import java.nio.channels.ScatteringByteChannel;
/** /**
@ -22,24 +23,42 @@ import java.nio.channels.ScatteringByteChannel;
*/ */
public class NetworkReceive implements Receive { public class NetworkReceive implements Receive {
private final int source; public final static String UNKNOWN_SOURCE = "";
public final static int UNLIMITED = -1;
private final String source;
private final ByteBuffer size; private final ByteBuffer size;
private final int maxSize;
private ByteBuffer buffer; private ByteBuffer buffer;
public NetworkReceive(int source, ByteBuffer buffer) {
public NetworkReceive(String source, ByteBuffer buffer) {
this.source = source; this.source = source;
this.buffer = buffer; this.buffer = buffer;
this.size = null; this.size = null;
this.maxSize = UNLIMITED;
} }
public NetworkReceive(int source) { public NetworkReceive(String source) {
this.source = source; this.source = source;
this.size = ByteBuffer.allocate(4); this.size = ByteBuffer.allocate(4);
this.buffer = null; this.buffer = null;
this.maxSize = UNLIMITED;
}
public NetworkReceive(int maxSize, String source) {
this.source = source;
this.size = ByteBuffer.allocate(4);
this.buffer = null;
this.maxSize = maxSize;
}
public NetworkReceive() {
this(UNKNOWN_SOURCE);
} }
@Override @Override
public int source() { public String source() {
return source; return source;
} }
@ -48,13 +67,15 @@ public class NetworkReceive implements Receive {
return !size.hasRemaining() && !buffer.hasRemaining(); return !size.hasRemaining() && !buffer.hasRemaining();
} }
@Override public long readFrom(ScatteringByteChannel channel) throws IOException {
public ByteBuffer[] reify() { return readFromReadableChannel(channel);
return new ByteBuffer[] {this.buffer};
} }
@Override // Need a method to read from ReadableByteChannel because BlockingChannel requires read with timeout
public long readFrom(ScatteringByteChannel channel) throws IOException { // 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; int read = 0;
if (size.hasRemaining()) { if (size.hasRemaining()) {
int bytesRead = channel.read(size); int bytesRead = channel.read(size);
@ -63,10 +84,12 @@ public class NetworkReceive implements Receive {
read += bytesRead; read += bytesRead;
if (!size.hasRemaining()) { if (!size.hasRemaining()) {
size.rewind(); size.rewind();
int requestSize = size.getInt(); int receiveSize = size.getInt();
if (requestSize < 0) if (receiveSize < 0)
throw new IllegalStateException("Invalid request (size = " + requestSize + ")"); throw new InvalidReceiveException("Invalid receive (size = " + receiveSize + ")");
this.buffer = ByteBuffer.allocate(requestSize); if (maxSize != UNLIMITED && receiveSize > maxSize)
throw new InvalidReceiveException("Invalid receive (size = " + receiveSize + " larger than " + maxSize + ")");
this.buffer = ByteBuffer.allocate(receiveSize);
} }
} }
if (buffer != null) { if (buffer != null) {
@ -83,4 +106,14 @@ public class NetworkReceive implements Receive {
return this.buffer; return this.buffer;
} }
// Used only by BlockingChannel, so we may be able to get rid of this when/if we get rid of BlockingChannel
@Deprecated
public long readCompletely(ReadableByteChannel channel) throws IOException {
int totalRead = 0;
while (!complete()) {
totalRead += readFromReadableChannel(channel);
}
return totalRead;
}
} }

View File

@ -23,7 +23,7 @@ import java.nio.ByteBuffer;
*/ */
public class NetworkSend extends ByteBufferSend { public class NetworkSend extends ByteBufferSend {
public NetworkSend(int destination, ByteBuffer... buffers) { public NetworkSend(String destination, ByteBuffer... buffers) {
super(destination, sizeDelimit(buffers)); super(destination, sizeDelimit(buffers));
} }

View File

@ -17,7 +17,6 @@
package org.apache.kafka.common.network; package org.apache.kafka.common.network;
import java.io.IOException; import java.io.IOException;
import java.nio.ByteBuffer;
import java.nio.channels.ScatteringByteChannel; import java.nio.channels.ScatteringByteChannel;
/** /**
@ -28,18 +27,13 @@ public interface Receive {
/** /**
* The numeric id of the source from which we are receiving data. * The numeric id of the source from which we are receiving data.
*/ */
public int source(); public String source();
/** /**
* Are we done receiving data? * Are we done receiving data?
*/ */
public boolean complete(); public boolean complete();
/**
* Turn this receive into ByteBuffer instances, if possible (otherwise returns null).
*/
public ByteBuffer[] reify();
/** /**
* Read bytes into this receive from the given channel * Read bytes into this receive from the given channel
* @param channel The channel to read from * @param channel The channel to read from

View File

@ -29,12 +29,12 @@ public interface Selectable {
* @param receiveBufferSize The receive buffer for the socket * @param receiveBufferSize The receive buffer for the socket
* @throws IOException If we cannot begin connecting * @throws IOException If we cannot begin connecting
*/ */
public void connect(int id, InetSocketAddress address, int sendBufferSize, int receiveBufferSize) throws IOException; public void connect(String id, InetSocketAddress address, int sendBufferSize, int receiveBufferSize) throws IOException;
/** /**
* Begin disconnecting the connection identified by the given id * Begin disconnecting the connection identified by the given id
*/ */
public void disconnect(int id); public void disconnect(String id);
/** /**
* Wakeup this selector if it is blocked on I/O * Wakeup this selector if it is blocked on I/O
@ -50,7 +50,7 @@ public interface Selectable {
* Queue the given request for sending in the subsequent {@poll(long)} calls * Queue the given request for sending in the subsequent {@poll(long)} calls
* @param send The request to send * @param send The request to send
*/ */
public void send(NetworkSend send); public void send(Send send);
/** /**
* Do I/O. Reads, writes, connection establishment, etc. * Do I/O. Reads, writes, connection establishment, etc.
@ -62,7 +62,7 @@ public interface Selectable {
/** /**
* The list of sends that completed on the last {@link #poll(long, List) poll()} call. * The list of sends that completed on the last {@link #poll(long, List) poll()} call.
*/ */
public List<NetworkSend> completedSends(); public List<Send> completedSends();
/** /**
* The list of receives that completed on the last {@link #poll(long, List) poll()} call. * The list of receives that completed on the last {@link #poll(long, List) poll()} call.
@ -73,25 +73,25 @@ public interface Selectable {
* The list of connections that finished disconnecting on the last {@link #poll(long, List) poll()} * The list of connections that finished disconnecting on the last {@link #poll(long, List) poll()}
* call. * call.
*/ */
public List<Integer> disconnected(); public List<String> disconnected();
/** /**
* The list of connections that completed their connection on the last {@link #poll(long, List) poll()} * The list of connections that completed their connection on the last {@link #poll(long, List) poll()}
* call. * call.
*/ */
public List<Integer> connected(); public List<String> connected();
/** /**
* Disable reads from the given connection * Disable reads from the given connection
* @param id The id for the connection * @param id The id for the connection
*/ */
public void mute(int id); public void mute(String id);
/** /**
* Re-enable reads from the given connection * Re-enable reads from the given connection
* @param id The id for the connection * @param id The id for the connection
*/ */
public void unmute(int id); public void unmute(String id);
/** /**
* Disable reads from all connections * Disable reads from all connections

View File

@ -17,17 +17,8 @@ import java.io.IOException;
import java.net.ConnectException; import java.net.ConnectException;
import java.net.InetSocketAddress; import java.net.InetSocketAddress;
import java.net.Socket; import java.net.Socket;
import java.nio.channels.CancelledKeyException; import java.nio.channels.*;
import java.nio.channels.SelectionKey; import java.util.*;
import java.nio.channels.SocketChannel;
import java.nio.channels.UnresolvedAddressException;
import java.util.ArrayList;
import java.util.HashMap;
import java.util.Iterator;
import java.util.LinkedHashMap;
import java.util.List;
import java.util.Map;
import java.util.Set;
import java.util.concurrent.TimeUnit; import java.util.concurrent.TimeUnit;
import org.apache.kafka.common.KafkaException; import org.apache.kafka.common.KafkaException;
@ -40,20 +31,21 @@ import org.apache.kafka.common.metrics.stats.Avg;
import org.apache.kafka.common.metrics.stats.Count; import org.apache.kafka.common.metrics.stats.Count;
import org.apache.kafka.common.metrics.stats.Max; import org.apache.kafka.common.metrics.stats.Max;
import org.apache.kafka.common.metrics.stats.Rate; import org.apache.kafka.common.metrics.stats.Rate;
import org.apache.kafka.common.utils.SystemTime;
import org.apache.kafka.common.utils.Time; import org.apache.kafka.common.utils.Time;
import org.slf4j.Logger; import org.slf4j.Logger;
import org.slf4j.LoggerFactory; import org.slf4j.LoggerFactory;
/** /**
* A selector interface for doing non-blocking multi-connection network I/O. * A nioSelector interface for doing non-blocking multi-connection network I/O.
* <p> * <p>
* This class works with {@link NetworkSend} and {@link NetworkReceive} to transmit size-delimited network requests and * This class works with {@link NetworkSend} and {@link NetworkReceive} to transmit size-delimited network requests and
* responses. * responses.
* <p> * <p>
* A connection can be added to the selector associated with an integer id by doing * A connection can be added to the nioSelector associated with an integer id by doing
* *
* <pre> * <pre>
* selector.connect(42, new InetSocketAddress(&quot;google.com&quot;, server.port), 64000, 64000); * nioSelector.connect(42, new InetSocketAddress(&quot;google.com&quot;, server.port), 64000, 64000);
* </pre> * </pre>
* *
* The connect call does not block on the creation of the TCP connection, so the connect method only begins initiating * The connect call does not block on the creation of the TCP connection, so the connect method only begins initiating
@ -64,10 +56,10 @@ import org.slf4j.LoggerFactory;
* *
* <pre> * <pre>
* List&lt;NetworkRequest&gt; requestsToSend = Arrays.asList(new NetworkRequest(0, myBytes), new NetworkRequest(1, myOtherBytes)); * List&lt;NetworkRequest&gt; requestsToSend = Arrays.asList(new NetworkRequest(0, myBytes), new NetworkRequest(1, myOtherBytes));
* selector.poll(TIMEOUT_MS, requestsToSend); * nioSelector.poll(TIMEOUT_MS, requestsToSend);
* </pre> * </pre>
* *
* The selector maintains several lists that are reset by each call to <code>poll()</code> which are available via * The nioSelector maintains several lists that are reset by each call to <code>poll()</code> which are available via
* various getters. These are reset by each call to <code>poll()</code>. * various getters. These are reset by each call to <code>poll()</code>.
* *
* This class is not thread safe! * This class is not thread safe!
@ -76,41 +68,59 @@ public class Selector implements Selectable {
private static final Logger log = LoggerFactory.getLogger(Selector.class); private static final Logger log = LoggerFactory.getLogger(Selector.class);
private final java.nio.channels.Selector selector; private final java.nio.channels.Selector nioSelector;
private final Map<Integer, SelectionKey> keys; private final Map<String, SelectionKey> keys;
private final List<NetworkSend> completedSends; private final List<Send> completedSends;
private final List<NetworkReceive> completedReceives; private final List<NetworkReceive> completedReceives;
private final List<Integer> disconnected; private final List<String> disconnected;
private final List<Integer> connected; private final List<String> connected;
private final List<Integer> failedSends; private final List<String> failedSends;
private final Time time; private final Time time;
private final SelectorMetrics sensors; private final SelectorMetrics sensors;
private final String metricGrpPrefix; private final String metricGrpPrefix;
private final Map<String, String> metricTags; private final Map<String, String> metricTags;
private final Map<String, Long> lruConnections;
private final long connectionsMaxIdleNanos;
private final int maxReceiveSize;
private final boolean metricsPerConnection;
private long currentTimeNanos;
private long nextIdleCloseCheckTime;
/** /**
* Create a new selector * Create a new nioSelector
*/ */
public Selector(Metrics metrics, Time time, String metricGrpPrefix, Map<String, String> metricTags) { public Selector(int maxReceiveSize, long connectionMaxIdleMs, Metrics metrics, Time time, String metricGrpPrefix, Map<String, String> metricTags, boolean metricsPerConnection) {
try { try {
this.selector = java.nio.channels.Selector.open(); this.nioSelector = java.nio.channels.Selector.open();
} catch (IOException e) { } catch (IOException e) {
throw new KafkaException(e); throw new KafkaException(e);
} }
this.maxReceiveSize = maxReceiveSize;
this.connectionsMaxIdleNanos = connectionMaxIdleMs * 1000 * 1000;
this.time = time; this.time = time;
this.metricGrpPrefix = metricGrpPrefix; this.metricGrpPrefix = metricGrpPrefix;
this.metricTags = metricTags; this.metricTags = metricTags;
this.keys = new HashMap<Integer, SelectionKey>(); this.keys = new HashMap<String, SelectionKey>();
this.completedSends = new ArrayList<NetworkSend>(); this.completedSends = new ArrayList<Send>();
this.completedReceives = new ArrayList<NetworkReceive>(); this.completedReceives = new ArrayList<NetworkReceive>();
this.connected = new ArrayList<Integer>(); this.connected = new ArrayList<String>();
this.disconnected = new ArrayList<Integer>(); this.disconnected = new ArrayList<String>();
this.failedSends = new ArrayList<Integer>(); this.failedSends = new ArrayList<String>();
this.sensors = new SelectorMetrics(metrics); this.sensors = new SelectorMetrics(metrics);
// initial capacity and load factor are default, we set them explicitly because we want to set accessOrder = true
this.lruConnections = new LinkedHashMap<String, Long>(16, .75F, true);
currentTimeNanos = new SystemTime().nanoseconds();
nextIdleCloseCheckTime = currentTimeNanos + connectionsMaxIdleNanos;
this.metricsPerConnection = metricsPerConnection;
}
public Selector(long connectionMaxIdleMS, Metrics metrics, Time time, String metricGrpPrefix, Map<String, String> metricTags) {
this(NetworkReceive.UNLIMITED, connectionMaxIdleMS, metrics, time, metricGrpPrefix, metricTags, true);
} }
/** /**
* Begin connecting to the given address and add the connection to this selector associated with the given id * Begin connecting to the given address and add the connection to this nioSelector associated with the given id
* number. * number.
* <p> * <p>
* Note that this call only initiates the connection, which will be completed on a future {@link #poll(long, List)} * Note that this call only initiates the connection, which will be completed on a future {@link #poll(long, List)}
@ -123,7 +133,7 @@ public class Selector implements Selectable {
* @throws IOException if DNS resolution fails on the hostname or if the broker is down * @throws IOException if DNS resolution fails on the hostname or if the broker is down
*/ */
@Override @Override
public void connect(int id, InetSocketAddress address, int sendBufferSize, int receiveBufferSize) throws IOException { public void connect(String id, InetSocketAddress address, int sendBufferSize, int receiveBufferSize) throws IOException {
if (this.keys.containsKey(id)) if (this.keys.containsKey(id))
throw new IllegalStateException("There is already a connection for id " + id); throw new IllegalStateException("There is already a connection for id " + id);
@ -143,7 +153,18 @@ public class Selector implements Selectable {
channel.close(); channel.close();
throw e; throw e;
} }
SelectionKey key = channel.register(this.selector, SelectionKey.OP_CONNECT); SelectionKey key = channel.register(this.nioSelector, SelectionKey.OP_CONNECT);
key.attach(new Transmissions(id));
this.keys.put(id, key);
}
/**
* Register the nioSelector with an existing channel
* Use this on server-side, when a connection is accepted by a different thread but processed by the Selector
* Note that we are not checking if the connection id is valid - since the connection already exists
*/
public void register(String id, SocketChannel channel) throws ClosedChannelException {
SelectionKey key = channel.register(nioSelector, SelectionKey.OP_READ);
key.attach(new Transmissions(id)); key.attach(new Transmissions(id));
this.keys.put(id, key); this.keys.put(id, key);
} }
@ -153,18 +174,18 @@ public class Selector implements Selectable {
* processed until the next {@link #poll(long, List) poll()} call. * processed until the next {@link #poll(long, List) poll()} call.
*/ */
@Override @Override
public void disconnect(int id) { public void disconnect(String id) {
SelectionKey key = this.keys.get(id); SelectionKey key = this.keys.get(id);
if (key != null) if (key != null)
key.cancel(); key.cancel();
} }
/** /**
* Interrupt the selector if it is blocked waiting to do I/O. * Interrupt the nioSelector if it is blocked waiting to do I/O.
*/ */
@Override @Override
public void wakeup() { public void wakeup() {
this.selector.wakeup(); this.nioSelector.wakeup();
} }
/** /**
@ -172,12 +193,14 @@ public class Selector implements Selectable {
*/ */
@Override @Override
public void close() { public void close() {
for (SelectionKey key : this.selector.keys()) List<String> connections = new LinkedList<String>(keys.keySet());
close(key); for (String id: connections)
close(id);
try { try {
this.selector.close(); this.nioSelector.close();
} catch (IOException e) { } catch (IOException e) {
log.error("Exception closing selector:", e); log.error("Exception closing nioSelector:", e);
} }
} }
@ -185,7 +208,7 @@ public class Selector implements Selectable {
* Queue the given request for sending in the subsequent {@poll(long)} calls * Queue the given request for sending in the subsequent {@poll(long)} calls
* @param send The request to send * @param send The request to send
*/ */
public void send(NetworkSend send) { public void send(Send send) {
SelectionKey key = keyForId(send.destination()); SelectionKey key = keyForId(send.destination());
Transmissions transmissions = transmissions(key); Transmissions transmissions = transmissions(key);
if (transmissions.hasSend()) if (transmissions.hasSend())
@ -194,7 +217,7 @@ public class Selector implements Selectable {
try { try {
key.interestOps(key.interestOps() | SelectionKey.OP_WRITE); key.interestOps(key.interestOps() | SelectionKey.OP_WRITE);
} catch (CancelledKeyException e) { } catch (CancelledKeyException e) {
close(key); close(transmissions.id);
this.failedSends.add(send.destination()); this.failedSends.add(send.destination());
} }
} }
@ -220,10 +243,11 @@ public class Selector implements Selectable {
long startSelect = time.nanoseconds(); long startSelect = time.nanoseconds();
int readyKeys = select(timeout); int readyKeys = select(timeout);
long endSelect = time.nanoseconds(); long endSelect = time.nanoseconds();
currentTimeNanos = endSelect;
this.sensors.selectTime.record(endSelect - startSelect, time.milliseconds()); this.sensors.selectTime.record(endSelect - startSelect, time.milliseconds());
if (readyKeys > 0) { if (readyKeys > 0) {
Set<SelectionKey> keys = this.selector.selectedKeys(); Set<SelectionKey> keys = this.nioSelector.selectedKeys();
Iterator<SelectionKey> iter = keys.iterator(); Iterator<SelectionKey> iter = keys.iterator();
while (iter.hasNext()) { while (iter.hasNext()) {
SelectionKey key = iter.next(); SelectionKey key = iter.next();
@ -232,8 +256,9 @@ public class Selector implements Selectable {
Transmissions transmissions = transmissions(key); Transmissions transmissions = transmissions(key);
SocketChannel channel = channel(key); SocketChannel channel = channel(key);
// register all per-broker metrics at once // register all per-connection metrics at once
sensors.maybeRegisterNodeMetrics(transmissions.id); sensors.maybeRegisterConnectionMetrics(transmissions.id);
lruConnections.put(transmissions.id, currentTimeNanos);
try { try {
/* complete any connections that have finished their handshake */ /* complete any connections that have finished their handshake */
@ -247,8 +272,14 @@ public class Selector implements Selectable {
/* read from any connections that have readable data */ /* read from any connections that have readable data */
if (key.isReadable()) { if (key.isReadable()) {
if (!transmissions.hasReceive()) if (!transmissions.hasReceive())
transmissions.receive = new NetworkReceive(transmissions.id); transmissions.receive = new NetworkReceive(maxReceiveSize, transmissions.id);
transmissions.receive.readFrom(channel); try {
transmissions.receive.readFrom(channel);
} catch (InvalidReceiveException e) {
log.error("Invalid data received from " + transmissions.id + " closing connection", e);
close(transmissions.id);
throw e;
}
if (transmissions.receive.complete()) { if (transmissions.receive.complete()) {
transmissions.receive.payload().rewind(); transmissions.receive.payload().rewind();
this.completedReceives.add(transmissions.receive); this.completedReceives.add(transmissions.receive);
@ -260,7 +291,7 @@ public class Selector implements Selectable {
/* write to any sockets that have space in their buffer and for which we have data */ /* write to any sockets that have space in their buffer and for which we have data */
if (key.isWritable()) { if (key.isWritable()) {
transmissions.send.writeTo(channel); transmissions.send.writeTo(channel);
if (transmissions.send.remaining() <= 0) { if (transmissions.send.completed()) {
this.completedSends.add(transmissions.send); this.completedSends.add(transmissions.send);
this.sensors.recordBytesSent(transmissions.id, transmissions.send.size()); this.sensors.recordBytesSent(transmissions.id, transmissions.send.size());
transmissions.clearSend(); transmissions.clearSend();
@ -270,7 +301,7 @@ public class Selector implements Selectable {
/* cancel any defunct sockets */ /* cancel any defunct sockets */
if (!key.isValid()) { if (!key.isValid()) {
close(key); close(transmissions.id);
this.disconnected.add(transmissions.id); this.disconnected.add(transmissions.id);
} }
} catch (IOException e) { } catch (IOException e) {
@ -279,15 +310,16 @@ public class Selector implements Selectable {
log.info("Connection {} disconnected", desc); log.info("Connection {} disconnected", desc);
else else
log.warn("Error in I/O with connection to {}", desc, e); log.warn("Error in I/O with connection to {}", desc, e);
close(key); close(transmissions.id);
this.disconnected.add(transmissions.id); this.disconnected.add(transmissions.id);
} }
} }
} }
long endIo = time.nanoseconds(); long endIo = time.nanoseconds();
this.sensors.ioTime.record(endIo - endSelect, time.milliseconds()); this.sensors.ioTime.record(endIo - endSelect, time.milliseconds());
maybeCloseOldestConnection();
} }
private String socketDescription(SocketChannel channel) { private String socketDescription(SocketChannel channel) {
Socket socket = channel.socket(); Socket socket = channel.socket();
if (socket == null) if (socket == null)
@ -299,7 +331,7 @@ public class Selector implements Selectable {
} }
@Override @Override
public List<NetworkSend> completedSends() { public List<Send> completedSends() {
return this.completedSends; return this.completedSends;
} }
@ -309,17 +341,17 @@ public class Selector implements Selectable {
} }
@Override @Override
public List<Integer> disconnected() { public List<String> disconnected() {
return this.disconnected; return this.disconnected;
} }
@Override @Override
public List<Integer> connected() { public List<String> connected() {
return this.connected; return this.connected;
} }
@Override @Override
public void mute(int id) { public void mute(String id) {
mute(this.keyForId(id)); mute(this.keyForId(id));
} }
@ -328,7 +360,7 @@ public class Selector implements Selectable {
} }
@Override @Override
public void unmute(int id) { public void unmute(String id) {
unmute(this.keyForId(id)); unmute(this.keyForId(id));
} }
@ -348,6 +380,27 @@ public class Selector implements Selectable {
unmute(key); unmute(key);
} }
private void maybeCloseOldestConnection() {
if (currentTimeNanos > nextIdleCloseCheckTime) {
if (lruConnections.isEmpty()) {
nextIdleCloseCheckTime = currentTimeNanos + connectionsMaxIdleNanos;
} else {
Map.Entry<String, Long> oldestConnectionEntry = lruConnections.entrySet().iterator().next();
Long connectionLastActiveTime = oldestConnectionEntry.getValue();
nextIdleCloseCheckTime = connectionLastActiveTime + connectionsMaxIdleNanos;
if (currentTimeNanos > nextIdleCloseCheckTime) {
String connectionId = oldestConnectionEntry.getKey();
if (log.isTraceEnabled())
log.trace("About to close the idle connection from " + connectionId
+ " due to being idle for " + (currentTimeNanos - connectionLastActiveTime) / 1000 / 1000 + " millis");
disconnected.add(connectionId);
close(connectionId);
}
}
}
}
/** /**
* Clear the results from the prior poll * Clear the results from the prior poll
*/ */
@ -369,17 +422,19 @@ public class Selector implements Selectable {
*/ */
private int select(long ms) throws IOException { private int select(long ms) throws IOException {
if (ms == 0L) if (ms == 0L)
return this.selector.selectNow(); return this.nioSelector.selectNow();
else if (ms < 0L) else if (ms < 0L)
return this.selector.select(); return this.nioSelector.select();
else else
return this.selector.select(ms); return this.nioSelector.select(ms);
} }
/** /**
* Begin closing this connection * Begin closing this connection
*/ */
private void close(SelectionKey key) { public void close(String id) {
SelectionKey key = keyForId(id);
lruConnections.remove(id);
SocketChannel channel = channel(key); SocketChannel channel = channel(key);
Transmissions trans = transmissions(key); Transmissions trans = transmissions(key);
if (trans != null) { if (trans != null) {
@ -401,10 +456,10 @@ public class Selector implements Selectable {
/** /**
* Get the selection key associated with this numeric id * Get the selection key associated with this numeric id
*/ */
private SelectionKey keyForId(int id) { private SelectionKey keyForId(String id) {
SelectionKey key = this.keys.get(id); SelectionKey key = this.keys.get(id);
if (key == null) if (key == null)
throw new IllegalStateException("Attempt to write to socket for which there is no open connection."); throw new IllegalStateException("Attempt to write to socket for which there is no open connection. Connection id " + id + " existing connections " + keys.keySet().toString());
return key; return key;
} }
@ -426,11 +481,11 @@ public class Selector implements Selectable {
* The id and in-progress send and receive associated with a connection * The id and in-progress send and receive associated with a connection
*/ */
private static class Transmissions { private static class Transmissions {
public int id; public String id;
public NetworkSend send; public Send send;
public NetworkReceive receive; public NetworkReceive receive;
public Transmissions(int id) { public Transmissions(String id) {
this.id = id; this.id = id;
} }
@ -464,20 +519,27 @@ public class Selector implements Selectable {
public SelectorMetrics(Metrics metrics) { public SelectorMetrics(Metrics metrics) {
this.metrics = metrics; this.metrics = metrics;
String metricGrpName = metricGrpPrefix + "-metrics"; String metricGrpName = metricGrpPrefix + "-metrics";
StringBuilder tagsSuffix = new StringBuilder();
this.connectionClosed = this.metrics.sensor("connections-closed"); for (Map.Entry<String, String> tag: metricTags.entrySet()) {
tagsSuffix.append(tag.getKey());
tagsSuffix.append("-");
tagsSuffix.append(tag.getValue());
}
this.connectionClosed = this.metrics.sensor("connections-closed:" + tagsSuffix.toString());
MetricName metricName = new MetricName("connection-close-rate", metricGrpName, "Connections closed per second in the window.", metricTags); MetricName metricName = new MetricName("connection-close-rate", metricGrpName, "Connections closed per second in the window.", metricTags);
this.connectionClosed.add(metricName, new Rate()); this.connectionClosed.add(metricName, new Rate());
this.connectionCreated = this.metrics.sensor("connections-created"); this.connectionCreated = this.metrics.sensor("connections-created:" + tagsSuffix.toString());
metricName = new MetricName("connection-creation-rate", metricGrpName, "New connections established per second in the window.", metricTags); metricName = new MetricName("connection-creation-rate", metricGrpName, "New connections established per second in the window.", metricTags);
this.connectionCreated.add(metricName, new Rate()); this.connectionCreated.add(metricName, new Rate());
this.bytesTransferred = this.metrics.sensor("bytes-sent-received"); this.bytesTransferred = this.metrics.sensor("bytes-sent-received:" + tagsSuffix.toString());
metricName = new MetricName("network-io-rate", metricGrpName, "The average number of network operations (reads or writes) on all connections per second.", metricTags); metricName = new MetricName("network-io-rate", metricGrpName, "The average number of network operations (reads or writes) on all connections per second.", metricTags);
bytesTransferred.add(metricName, new Rate(new Count())); bytesTransferred.add(metricName, new Rate(new Count()));
this.bytesSent = this.metrics.sensor("bytes-sent", bytesTransferred); this.bytesSent = this.metrics.sensor("bytes-sent:" + tagsSuffix.toString(), bytesTransferred);
metricName = new MetricName("outgoing-byte-rate", metricGrpName, "The average number of outgoing bytes sent per second to all servers.", metricTags); metricName = new MetricName("outgoing-byte-rate", metricGrpName, "The average number of outgoing bytes sent per second to all servers.", metricTags);
this.bytesSent.add(metricName, new Rate()); this.bytesSent.add(metricName, new Rate());
metricName = new MetricName("request-rate", metricGrpName, "The average number of requests sent per second.", metricTags); metricName = new MetricName("request-rate", metricGrpName, "The average number of requests sent per second.", metricTags);
@ -487,13 +549,13 @@ public class Selector implements Selectable {
metricName = new MetricName("request-size-max", metricGrpName, "The maximum size of any request sent in the window.", metricTags); metricName = new MetricName("request-size-max", metricGrpName, "The maximum size of any request sent in the window.", metricTags);
this.bytesSent.add(metricName, new Max()); this.bytesSent.add(metricName, new Max());
this.bytesReceived = this.metrics.sensor("bytes-received", bytesTransferred); this.bytesReceived = this.metrics.sensor("bytes-received:" + tagsSuffix.toString(), bytesTransferred);
metricName = new MetricName("incoming-byte-rate", metricGrpName, "Bytes/second read off all sockets", metricTags); metricName = new MetricName("incoming-byte-rate", metricGrpName, "Bytes/second read off all sockets", metricTags);
this.bytesReceived.add(metricName, new Rate()); this.bytesReceived.add(metricName, new Rate());
metricName = new MetricName("response-rate", metricGrpName, "Responses received sent per second.", metricTags); metricName = new MetricName("response-rate", metricGrpName, "Responses received sent per second.", metricTags);
this.bytesReceived.add(metricName, new Rate(new Count())); this.bytesReceived.add(metricName, new Rate(new Count()));
this.selectTime = this.metrics.sensor("select-time"); this.selectTime = this.metrics.sensor("select-time:" + tagsSuffix.toString());
metricName = new MetricName("select-rate", metricGrpName, "Number of times the I/O layer checked for new I/O to perform per second", metricTags); metricName = new MetricName("select-rate", metricGrpName, "Number of times the I/O layer checked for new I/O to perform per second", metricTags);
this.selectTime.add(metricName, new Rate(new Count())); this.selectTime.add(metricName, new Rate(new Count()));
metricName = new MetricName("io-wait-time-ns-avg", metricGrpName, "The average length of time the I/O thread spent waiting for a socket ready for reads or writes in nanoseconds.", metricTags); metricName = new MetricName("io-wait-time-ns-avg", metricGrpName, "The average length of time the I/O thread spent waiting for a socket ready for reads or writes in nanoseconds.", metricTags);
@ -501,7 +563,7 @@ public class Selector implements Selectable {
metricName = new MetricName("io-wait-ratio", metricGrpName, "The fraction of time the I/O thread spent waiting.", metricTags); metricName = new MetricName("io-wait-ratio", metricGrpName, "The fraction of time the I/O thread spent waiting.", metricTags);
this.selectTime.add(metricName, new Rate(TimeUnit.NANOSECONDS)); this.selectTime.add(metricName, new Rate(TimeUnit.NANOSECONDS));
this.ioTime = this.metrics.sensor("io-time"); this.ioTime = this.metrics.sensor("io-time:" + tagsSuffix.toString());
metricName = new MetricName("io-time-ns-avg", metricGrpName, "The average length of time for I/O per select call in nanoseconds.", metricTags); metricName = new MetricName("io-time-ns-avg", metricGrpName, "The average length of time for I/O per select call in nanoseconds.", metricTags);
this.ioTime.add(metricName, new Avg()); this.ioTime.add(metricName, new Avg());
metricName = new MetricName("io-ratio", metricGrpName, "The fraction of time the I/O thread spent doing I/O", metricTags); metricName = new MetricName("io-ratio", metricGrpName, "The fraction of time the I/O thread spent doing I/O", metricTags);
@ -515,17 +577,17 @@ public class Selector implements Selectable {
}); });
} }
public void maybeRegisterNodeMetrics(int node) { public void maybeRegisterConnectionMetrics(String connectionId) {
if (node >= 0) { if (!connectionId.isEmpty() && metricsPerConnection) {
// if one sensor of the metrics has been registered for the node, // if one sensor of the metrics has been registered for the connection,
// then all other sensors should have been registered; and vice versa // then all other sensors should have been registered; and vice versa
String nodeRequestName = "node-" + node + ".bytes-sent"; String nodeRequestName = "node-" + connectionId + ".bytes-sent";
Sensor nodeRequest = this.metrics.getSensor(nodeRequestName); Sensor nodeRequest = this.metrics.getSensor(nodeRequestName);
if (nodeRequest == null) { if (nodeRequest == null) {
String metricGrpName = metricGrpPrefix + "-node-metrics"; String metricGrpName = metricGrpPrefix + "-node-metrics";
Map<String, String> tags = new LinkedHashMap<String, String>(metricTags); Map<String, String> tags = new LinkedHashMap<String, String>(metricTags);
tags.put("node-id", "node-" + node); tags.put("node-id", "node-" + connectionId);
nodeRequest = this.metrics.sensor(nodeRequestName); nodeRequest = this.metrics.sensor(nodeRequestName);
MetricName metricName = new MetricName("outgoing-byte-rate", metricGrpName, tags); MetricName metricName = new MetricName("outgoing-byte-rate", metricGrpName, tags);
@ -537,14 +599,14 @@ public class Selector implements Selectable {
metricName = new MetricName("request-size-max", metricGrpName, "The maximum size of any request sent in the window.", tags); metricName = new MetricName("request-size-max", metricGrpName, "The maximum size of any request sent in the window.", tags);
nodeRequest.add(metricName, new Max()); nodeRequest.add(metricName, new Max());
String nodeResponseName = "node-" + node + ".bytes-received"; String nodeResponseName = "node-" + connectionId + ".bytes-received";
Sensor nodeResponse = this.metrics.sensor(nodeResponseName); Sensor nodeResponse = this.metrics.sensor(nodeResponseName);
metricName = new MetricName("incoming-byte-rate", metricGrpName, tags); metricName = new MetricName("incoming-byte-rate", metricGrpName, tags);
nodeResponse.add(metricName, new Rate()); nodeResponse.add(metricName, new Rate());
metricName = new MetricName("response-rate", metricGrpName, "The average number of responses received per second.", tags); metricName = new MetricName("response-rate", metricGrpName, "The average number of responses received per second.", tags);
nodeResponse.add(metricName, new Rate(new Count())); nodeResponse.add(metricName, new Rate(new Count()));
String nodeTimeName = "node-" + node + ".latency"; String nodeTimeName = "node-" + connectionId + ".latency";
Sensor nodeRequestTime = this.metrics.sensor(nodeTimeName); Sensor nodeRequestTime = this.metrics.sensor(nodeTimeName);
metricName = new MetricName("request-latency-avg", metricGrpName, tags); metricName = new MetricName("request-latency-avg", metricGrpName, tags);
nodeRequestTime.add(metricName, new Avg()); nodeRequestTime.add(metricName, new Avg());
@ -554,22 +616,22 @@ public class Selector implements Selectable {
} }
} }
public void recordBytesSent(int node, int bytes) { public void recordBytesSent(String connectionId, long bytes) {
long now = time.milliseconds(); long now = time.milliseconds();
this.bytesSent.record(bytes, now); this.bytesSent.record(bytes, now);
if (node >= 0) { if (!connectionId.isEmpty()) {
String nodeRequestName = "node-" + node + ".bytes-sent"; String nodeRequestName = "node-" + connectionId + ".bytes-sent";
Sensor nodeRequest = this.metrics.getSensor(nodeRequestName); Sensor nodeRequest = this.metrics.getSensor(nodeRequestName);
if (nodeRequest != null) if (nodeRequest != null)
nodeRequest.record(bytes, now); nodeRequest.record(bytes, now);
} }
} }
public void recordBytesReceived(int node, int bytes) { public void recordBytesReceived(String connection, int bytes) {
long now = time.milliseconds(); long now = time.milliseconds();
this.bytesReceived.record(bytes, now); this.bytesReceived.record(bytes, now);
if (node >= 0) { if (!connection.isEmpty()) {
String nodeRequestName = "node-" + node + ".bytes-received"; String nodeRequestName = "node-" + connection + ".bytes-received";
Sensor nodeRequest = this.metrics.getSensor(nodeRequestName); Sensor nodeRequest = this.metrics.getSensor(nodeRequestName);
if (nodeRequest != null) if (nodeRequest != null)
nodeRequest.record(bytes, now); nodeRequest.record(bytes, now);

View File

@ -13,7 +13,6 @@
package org.apache.kafka.common.network; package org.apache.kafka.common.network;
import java.io.IOException; import java.io.IOException;
import java.nio.ByteBuffer;
import java.nio.channels.GatheringByteChannel; import java.nio.channels.GatheringByteChannel;
/** /**
@ -24,23 +23,13 @@ public interface Send {
/** /**
* The numeric id for the destination of this send * The numeric id for the destination of this send
*/ */
public int destination(); public String destination();
/**
* The number of bytes remaining to send
*/
public int remaining();
/** /**
* Is this send complete? * Is this send complete?
*/ */
public boolean completed(); public boolean completed();
/**
* An optional method to turn this send into an array of ByteBuffers if possible (otherwise returns null)
*/
public ByteBuffer[] reify();
/** /**
* Write some as-yet unwritten bytes from this send to the provided channel. It may take multiple calls for the send * Write some as-yet unwritten bytes from this send to the provided channel. It may take multiple calls for the send
* to be completely written * to be completely written
@ -50,4 +39,9 @@ public interface Send {
*/ */
public long writeTo(GatheringByteChannel channel) throws IOException; public long writeTo(GatheringByteChannel channel) throws IOException;
/**
* Size of the send
*/
public long size();
} }

View File

@ -69,7 +69,15 @@ public enum Errors {
INVALID_REQUIRED_ACKS(21, INVALID_REQUIRED_ACKS(21,
new InvalidRequiredAcksException("Produce request specified an invalid value for required acks.")), new InvalidRequiredAcksException("Produce request specified an invalid value for required acks.")),
ILLEGAL_GENERATION(22, ILLEGAL_GENERATION(22,
new ApiException("Specified consumer generation id is not valid.")); new ApiException("Specified consumer generation id is not valid.")),
INCONSISTENT_PARTITION_ASSIGNMENT_STRATEGY(23,
new ApiException("The request partition assignment strategy does not match that of the group.")),
UNKNOWN_PARTITION_ASSIGNMENT_STRATEGY(24,
new ApiException("The request partition assignment strategy is unknown to the broker.")),
UNKNOWN_CONSUMER_ID(25,
new ApiException("The coordinator is not aware of this consumer.")),
INVALID_SESSION_TIMEOUT(26,
new ApiException("The session timeout is not within an acceptable range."));
private static Map<Class<?>, Errors> classToError = new HashMap<Class<?>, Errors>(); private static Map<Class<?>, Errors> classToError = new HashMap<Class<?>, Errors>();
private static Map<Short, Errors> codeToError = new HashMap<Short, Errors>(); private static Map<Short, Errors> codeToError = new HashMap<Short, Errors>();

View File

@ -118,6 +118,16 @@ public class Protocol {
new Field("offset", new Field("offset",
INT64, INT64,
"Message offset to be committed."), "Message offset to be committed."),
new Field("metadata",
STRING,
"Any associated metadata the client wants to keep."));
public static final Schema OFFSET_COMMIT_REQUEST_PARTITION_V1 = new Schema(new Field("partition",
INT32,
"Topic partition id."),
new Field("offset",
INT64,
"Message offset to be committed."),
new Field("timestamp", new Field("timestamp",
INT64, INT64,
"Timestamp of the commit"), "Timestamp of the commit"),
@ -125,7 +135,7 @@ public class Protocol {
STRING, STRING,
"Any associated metadata the client wants to keep.")); "Any associated metadata the client wants to keep."));
public static final Schema OFFSET_COMMIT_REQUEST_PARTITION_V1 = new Schema(new Field("partition", public static final Schema OFFSET_COMMIT_REQUEST_PARTITION_V2 = new Schema(new Field("partition",
INT32, INT32,
"Topic partition id."), "Topic partition id."),
new Field("offset", new Field("offset",
@ -149,6 +159,13 @@ public class Protocol {
new ArrayOf(OFFSET_COMMIT_REQUEST_PARTITION_V1), new ArrayOf(OFFSET_COMMIT_REQUEST_PARTITION_V1),
"Partitions to commit offsets.")); "Partitions to commit offsets."));
public static final Schema OFFSET_COMMIT_REQUEST_TOPIC_V2 = new Schema(new Field("topic",
STRING,
"Topic to commit."),
new Field("partitions",
new ArrayOf(OFFSET_COMMIT_REQUEST_PARTITION_V2),
"Partitions to commit offsets."));
public static final Schema OFFSET_COMMIT_REQUEST_V0 = new Schema(new Field("group_id", public static final Schema OFFSET_COMMIT_REQUEST_V0 = new Schema(new Field("group_id",
STRING, STRING,
"The consumer group id."), "The consumer group id."),
@ -166,7 +183,7 @@ public class Protocol {
STRING, STRING,
"The consumer id assigned by the group coordinator."), "The consumer id assigned by the group coordinator."),
new Field("topics", new Field("topics",
new ArrayOf(OFFSET_COMMIT_REQUEST_TOPIC_V0), new ArrayOf(OFFSET_COMMIT_REQUEST_TOPIC_V1),
"Topics to commit offsets.")); "Topics to commit offsets."));
public static final Schema OFFSET_COMMIT_REQUEST_V2 = new Schema(new Field("group_id", public static final Schema OFFSET_COMMIT_REQUEST_V2 = new Schema(new Field("group_id",
@ -182,7 +199,7 @@ public class Protocol {
INT64, INT64,
"Time period in ms to retain the offset."), "Time period in ms to retain the offset."),
new Field("topics", new Field("topics",
new ArrayOf(OFFSET_COMMIT_REQUEST_TOPIC_V1), new ArrayOf(OFFSET_COMMIT_REQUEST_TOPIC_V2),
"Topics to commit offsets.")); "Topics to commit offsets."));
public static final Schema OFFSET_COMMIT_RESPONSE_PARTITION_V0 = new Schema(new Field("partition", public static final Schema OFFSET_COMMIT_RESPONSE_PARTITION_V0 = new Schema(new Field("partition",
@ -199,10 +216,20 @@ public class Protocol {
new ArrayOf(OFFSET_COMMIT_RESPONSE_TOPIC_V0))); new ArrayOf(OFFSET_COMMIT_RESPONSE_TOPIC_V0)));
public static final Schema[] OFFSET_COMMIT_REQUEST = new Schema[] {OFFSET_COMMIT_REQUEST_V0, OFFSET_COMMIT_REQUEST_V1, OFFSET_COMMIT_REQUEST_V2}; public static final Schema[] OFFSET_COMMIT_REQUEST = new Schema[] {OFFSET_COMMIT_REQUEST_V0, OFFSET_COMMIT_REQUEST_V1, OFFSET_COMMIT_REQUEST_V2};
/* The response types for V0, V1 and V2 of OFFSET_COMMIT_REQUEST are the same. */ /* The response types for V0, V1 and V2 of OFFSET_COMMIT_REQUEST are the same. */
public static final Schema[] OFFSET_COMMIT_RESPONSE = new Schema[] {OFFSET_COMMIT_RESPONSE_V0, OFFSET_COMMIT_RESPONSE_V0, OFFSET_COMMIT_RESPONSE_V0}; public static final Schema OFFSET_COMMIT_RESPONSE_V1 = OFFSET_COMMIT_RESPONSE_V0;
public static final Schema OFFSET_COMMIT_RESPONSE_V2 = OFFSET_COMMIT_RESPONSE_V0;
public static final Schema[] OFFSET_COMMIT_RESPONSE = new Schema[] {OFFSET_COMMIT_RESPONSE_V0, OFFSET_COMMIT_RESPONSE_V1, OFFSET_COMMIT_RESPONSE_V2};
/* Offset fetch api */ /* Offset fetch api */
/*
* Wire formats of version 0 and 1 are the same, but with different functionality.
* Version 0 will read the offsets from ZK;
* Version 1 will read the offsets from Kafka.
*/
public static final Schema OFFSET_FETCH_REQUEST_PARTITION_V0 = new Schema(new Field("partition", public static final Schema OFFSET_FETCH_REQUEST_PARTITION_V0 = new Schema(new Field("partition",
INT32, INT32,
"Topic partition id.")); "Topic partition id."));
@ -239,8 +266,11 @@ public class Protocol {
public static final Schema OFFSET_FETCH_RESPONSE_V0 = new Schema(new Field("responses", public static final Schema OFFSET_FETCH_RESPONSE_V0 = new Schema(new Field("responses",
new ArrayOf(OFFSET_FETCH_RESPONSE_TOPIC_V0))); new ArrayOf(OFFSET_FETCH_RESPONSE_TOPIC_V0)));
public static final Schema[] OFFSET_FETCH_REQUEST = new Schema[] {OFFSET_FETCH_REQUEST_V0}; public static final Schema OFFSET_FETCH_REQUEST_V1 = OFFSET_FETCH_REQUEST_V0;
public static final Schema[] OFFSET_FETCH_RESPONSE = new Schema[] {OFFSET_FETCH_RESPONSE_V0}; public static final Schema OFFSET_FETCH_RESPONSE_V1 = OFFSET_FETCH_RESPONSE_V0;
public static final Schema[] OFFSET_FETCH_REQUEST = new Schema[] {OFFSET_FETCH_REQUEST_V0, OFFSET_FETCH_REQUEST_V1};
public static final Schema[] OFFSET_FETCH_RESPONSE = new Schema[] {OFFSET_FETCH_RESPONSE_V0, OFFSET_FETCH_RESPONSE_V1};
/* List offset api */ /* List offset api */
public static final Schema LIST_OFFSET_REQUEST_PARTITION_V0 = new Schema(new Field("partition", public static final Schema LIST_OFFSET_REQUEST_PARTITION_V0 = new Schema(new Field("partition",

View File

@ -49,7 +49,7 @@ public enum SecurityProtocol {
} }
public static String getName(int id) { public static String getName(int id) {
return CODE_TO_SECURITY_PROTOCOL.get(id).name; return CODE_TO_SECURITY_PROTOCOL.get((short) id).name;
} }
public static List<String> getNames() { public static List<String> getNames() {

View File

@ -27,7 +27,10 @@ public class HeartbeatResponse extends AbstractRequestResponse {
/** /**
* Possible error code: * Possible error code:
* *
* TODO * CONSUMER_COORDINATOR_NOT_AVAILABLE (15)
* NOT_COORDINATOR_FOR_CONSUMER (16)
* ILLEGAL_GENERATION (22)
* UNKNOWN_CONSUMER_ID (25)
*/ */
private final short errorCode; private final short errorCode;

View File

@ -30,7 +30,12 @@ public class JoinGroupResponse extends AbstractRequestResponse {
/** /**
* Possible error code: * Possible error code:
* *
* TODO * CONSUMER_COORDINATOR_NOT_AVAILABLE (15)
* NOT_COORDINATOR_FOR_CONSUMER (16)
* INCONSISTENT_PARTITION_ASSIGNMENT_STRATEGY (23)
* UNKNOWN_PARTITION_ASSIGNMENT_STRATEGY (24)
* UNKNOWN_CONSUMER_ID (25)
* INVALID_SESSION_TIMEOUT (26)
*/ */
private static final String GENERATION_ID_KEY_NAME = "group_generation_id"; private static final String GENERATION_ID_KEY_NAME = "group_generation_id";

View File

@ -68,7 +68,7 @@ public class OffsetCommitRequest extends AbstractRequest {
public static final class PartitionData { public static final class PartitionData {
@Deprecated @Deprecated
public final long timestamp; // for V0, V1 public final long timestamp; // for V1
public final long offset; public final long offset;
public final String metadata; public final String metadata;
@ -93,6 +93,7 @@ public class OffsetCommitRequest extends AbstractRequest {
@Deprecated @Deprecated
public OffsetCommitRequest(String groupId, Map<TopicPartition, PartitionData> offsetData) { public OffsetCommitRequest(String groupId, Map<TopicPartition, PartitionData> offsetData) {
super(new Struct(ProtoUtils.requestSchema(ApiKeys.OFFSET_COMMIT.id, 0))); super(new Struct(ProtoUtils.requestSchema(ApiKeys.OFFSET_COMMIT.id, 0)));
initCommonFields(groupId, offsetData); initCommonFields(groupId, offsetData);
this.groupId = groupId; this.groupId = groupId;
this.generationId = DEFAULT_GENERATION_ID; this.generationId = DEFAULT_GENERATION_ID;
@ -159,7 +160,7 @@ public class OffsetCommitRequest extends AbstractRequest {
Struct partitionData = topicData.instance(PARTITIONS_KEY_NAME); Struct partitionData = topicData.instance(PARTITIONS_KEY_NAME);
partitionData.set(PARTITION_KEY_NAME, partitionEntry.getKey()); partitionData.set(PARTITION_KEY_NAME, partitionEntry.getKey());
partitionData.set(COMMIT_OFFSET_KEY_NAME, fetchPartitionData.offset); partitionData.set(COMMIT_OFFSET_KEY_NAME, fetchPartitionData.offset);
// Only for v0 and v1 // Only for v1
if (partitionData.hasField(TIMESTAMP_KEY_NAME)) if (partitionData.hasField(TIMESTAMP_KEY_NAME))
partitionData.set(TIMESTAMP_KEY_NAME, fetchPartitionData.timestamp); partitionData.set(TIMESTAMP_KEY_NAME, fetchPartitionData.timestamp);
partitionData.set(METADATA_KEY_NAME, fetchPartitionData.metadata); partitionData.set(METADATA_KEY_NAME, fetchPartitionData.metadata);
@ -203,7 +204,7 @@ public class OffsetCommitRequest extends AbstractRequest {
long offset = partitionDataStruct.getLong(COMMIT_OFFSET_KEY_NAME); long offset = partitionDataStruct.getLong(COMMIT_OFFSET_KEY_NAME);
String metadata = partitionDataStruct.getString(METADATA_KEY_NAME); String metadata = partitionDataStruct.getString(METADATA_KEY_NAME);
PartitionData partitionOffset; PartitionData partitionOffset;
// This field only exists in v0 and v1 // This field only exists in v1
if (partitionDataStruct.hasField(TIMESTAMP_KEY_NAME)) { if (partitionDataStruct.hasField(TIMESTAMP_KEY_NAME)) {
long timestamp = partitionDataStruct.getLong(TIMESTAMP_KEY_NAME); long timestamp = partitionDataStruct.getLong(TIMESTAMP_KEY_NAME);
partitionOffset = new PartitionData(offset, timestamp, metadata); partitionOffset = new PartitionData(offset, timestamp, metadata);

View File

@ -25,7 +25,7 @@ public class RequestSend extends NetworkSend {
private final RequestHeader header; private final RequestHeader header;
private final Struct body; private final Struct body;
public RequestSend(int destination, RequestHeader header, Struct body) { public RequestSend(String destination, RequestHeader header, Struct body) {
super(destination, serialize(header, body)); super(destination, serialize(header, body));
this.header = header; this.header = header;
this.body = body; this.body = body;

View File

@ -0,0 +1,41 @@
/**
* 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.requests;
import org.apache.kafka.common.network.NetworkSend;
import org.apache.kafka.common.protocol.types.Struct;
import java.nio.ByteBuffer;
public class ResponseSend extends NetworkSend {
public ResponseSend(String destination, ResponseHeader header, Struct body) {
super(destination, serialize(header, body));
}
public ResponseSend(String destination, ResponseHeader header, AbstractRequestResponse response) {
this(destination, header, response.toStruct());
}
private static ByteBuffer serialize(ResponseHeader header, Struct body) {
ByteBuffer buffer = ByteBuffer.allocate(header.sizeOf() + body.sizeOf());
header.writeTo(buffer);
body.writeTo(buffer);
buffer.rewind();
return buffer;
}
}

View File

@ -13,6 +13,7 @@
package org.apache.kafka.common.serialization; package org.apache.kafka.common.serialization;
import java.io.Closeable;
import java.util.Map; import java.util.Map;
/** /**
@ -21,7 +22,7 @@ import java.util.Map;
* *
* A class that implements this interface is expected to have a constructor with no parameter. * A class that implements this interface is expected to have a constructor with no parameter.
*/ */
public interface Deserializer<T> { public interface Deserializer<T> extends Closeable {
/** /**
* Configure this class. * Configure this class.
@ -38,8 +39,6 @@ public interface Deserializer<T> {
*/ */
public T deserialize(String topic, byte[] data); public T deserialize(String topic, byte[] data);
/** @Override
* Close this deserializer
*/
public void close(); public void close();
} }

View File

@ -0,0 +1,44 @@
/**
* 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.serialization;
import org.apache.kafka.common.errors.SerializationException;
import java.util.Map;
public class IntegerDeserializer implements Deserializer<Integer> {
public void configure(Map<String, ?> configs, boolean isKey) {
// nothing to do
}
public Integer deserialize(String topic, byte[] data) {
if (data == null)
return null;
if (data.length != 4) {
throw new SerializationException("Size of data received by IntegerDeserializer is " +
"not 4");
}
int value = 0;
for (byte b : data) {
value <<= 8;
value |= b & 0xFF;
}
return value;
}
public void close() {
// nothing to do
}
}

View File

@ -0,0 +1,38 @@
/**
* 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.serialization;
import java.util.Map;
public class IntegerSerializer implements Serializer<Integer> {
public void configure(Map<String, ?> configs, boolean isKey) {
// nothing to do
}
public byte[] serialize(String topic, Integer data) {
if (data == null)
return null;
return new byte[] {
(byte) (data >>> 24),
(byte) (data >>> 16),
(byte) (data >>> 8),
data.byteValue()
};
}
public void close() {
// nothing to do
}
}

View File

@ -13,6 +13,7 @@
package org.apache.kafka.common.serialization; package org.apache.kafka.common.serialization;
import java.io.Closeable;
import java.util.Map; import java.util.Map;
/** /**
@ -21,7 +22,7 @@ import java.util.Map;
* *
* A class that implements this interface is expected to have a constructor with no parameter. * A class that implements this interface is expected to have a constructor with no parameter.
*/ */
public interface Serializer<T> { public interface Serializer<T> extends Closeable {
/** /**
* Configure this class. * Configure this class.
@ -37,8 +38,12 @@ public interface Serializer<T> {
*/ */
public byte[] serialize(String topic, T data); public byte[] serialize(String topic, T data);
/** /**
* Close this serializer * Close this serializer.
* This method has to be idempotent if the serializer is used in KafkaProducer because it might be called
* multiple times.
*/ */
@Override
public void close(); public void close();
} }

View File

@ -85,6 +85,20 @@ public class MetadataTest {
} }
} }
@Test
public void testFailedUpdate() {
long time = 100;
metadata.update(Cluster.empty(), time);
assertEquals(100, metadata.timeToNextUpdate(1000));
metadata.failedUpdate(1100);
assertEquals(100, metadata.timeToNextUpdate(1100));
assertEquals(100, metadata.lastSuccessfulUpdate());
}
private Thread asyncFetch(final String topic) { private Thread asyncFetch(final String topic) {
Thread thread = new Thread() { Thread thread = new Thread() {
public void run() { public void run() {

View File

@ -78,7 +78,7 @@ public class MockClient implements KafkaClient {
return false; return false;
} }
public void disconnect(Integer node) { public void disconnect(String node) {
Iterator<ClientRequest> iter = requests.iterator(); Iterator<ClientRequest> iter = requests.iterator();
while (iter.hasNext()) { while (iter.hasNext()) {
ClientRequest request = iter.next(); ClientRequest request = iter.next();
@ -115,7 +115,7 @@ public class MockClient implements KafkaClient {
} }
@Override @Override
public List<ClientResponse> completeAll(int node, long now) { public List<ClientResponse> completeAll(String node, long now) {
return completeAll(now); return completeAll(now);
} }
@ -158,7 +158,7 @@ public class MockClient implements KafkaClient {
} }
@Override @Override
public int inFlightRequestCount(int nodeId) { public int inFlightRequestCount(String nodeId) {
return requests.size(); return requests.size();
} }

View File

@ -65,7 +65,7 @@ public class NetworkClientTest {
client.poll(1, time.milliseconds()); client.poll(1, time.milliseconds());
selector.clear(); selector.clear();
assertTrue("Now the client is ready", client.ready(node, time.milliseconds())); assertTrue("Now the client is ready", client.ready(node, time.milliseconds()));
selector.disconnect(node.id()); selector.disconnect(node.idString());
client.poll(1, time.milliseconds()); client.poll(1, time.milliseconds());
selector.clear(); selector.clear();
assertFalse("After we forced the disconnection the client is no longer ready.", client.ready(node, time.milliseconds())); assertFalse("After we forced the disconnection the client is no longer ready.", client.ready(node, time.milliseconds()));
@ -74,7 +74,7 @@ public class NetworkClientTest {
@Test(expected = IllegalStateException.class) @Test(expected = IllegalStateException.class)
public void testSendToUnreadyNode() { public void testSendToUnreadyNode() {
RequestSend send = new RequestSend(5, RequestSend send = new RequestSend("5",
client.nextRequestHeader(ApiKeys.METADATA), client.nextRequestHeader(ApiKeys.METADATA),
new MetadataRequest(Arrays.asList("test")).toStruct()); new MetadataRequest(Arrays.asList("test")).toStruct());
ClientRequest request = new ClientRequest(time.milliseconds(), false, send, null); ClientRequest request = new ClientRequest(time.milliseconds(), false, send, null);
@ -86,7 +86,7 @@ public class NetworkClientTest {
public void testSimpleRequestResponse() { public void testSimpleRequestResponse() {
ProduceRequest produceRequest = new ProduceRequest((short) 1, 1000, Collections.<TopicPartition, ByteBuffer>emptyMap()); ProduceRequest produceRequest = new ProduceRequest((short) 1, 1000, Collections.<TopicPartition, ByteBuffer>emptyMap());
RequestHeader reqHeader = client.nextRequestHeader(ApiKeys.PRODUCE); RequestHeader reqHeader = client.nextRequestHeader(ApiKeys.PRODUCE);
RequestSend send = new RequestSend(node.id(), reqHeader, produceRequest.toStruct()); RequestSend send = new RequestSend(node.idString(), reqHeader, produceRequest.toStruct());
TestCallbackHandler handler = new TestCallbackHandler(); TestCallbackHandler handler = new TestCallbackHandler();
ClientRequest request = new ClientRequest(time.milliseconds(), true, send, handler); ClientRequest request = new ClientRequest(time.milliseconds(), true, send, handler);
awaitReady(client, node); awaitReady(client, node);
@ -101,7 +101,7 @@ public class NetworkClientTest {
respHeader.writeTo(buffer); respHeader.writeTo(buffer);
resp.writeTo(buffer); resp.writeTo(buffer);
buffer.flip(); buffer.flip();
selector.completeReceive(new NetworkReceive(node.id(), buffer)); selector.completeReceive(new NetworkReceive(node.idString(), buffer));
List<ClientResponse> responses = client.poll(1, time.milliseconds()); List<ClientResponse> responses = client.poll(1, time.milliseconds());
assertEquals(1, responses.size()); assertEquals(1, responses.size());
assertTrue("The handler should have executed.", handler.executed); assertTrue("The handler should have executed.", handler.executed);

View File

@ -0,0 +1,49 @@
/**
* Licensed to the Apache Software Foundation (ASF) under one or more
* contributor license agreements. See the NOTICE file distributed with
* this work for additional information regarding copyright ownership.
* The ASF licenses this file to You under the Apache License, Version 2.0
* (the "License"); you may not use this file except in compliance with
* the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.kafka.clients.consumer;
import org.apache.kafka.common.KafkaException;
import org.apache.kafka.common.serialization.ByteArrayDeserializer;
import org.apache.kafka.test.MockMetricsReporter;
import org.junit.Assert;
import org.junit.Test;
import java.util.Properties;
public class KafkaConsumerTest {
@Test
public void testConstructorClose() throws Exception {
Properties props = new Properties();
props.setProperty(ConsumerConfig.CLIENT_ID_CONFIG, "testConstructorClose");
props.setProperty(ConsumerConfig.BOOTSTRAP_SERVERS_CONFIG, "some.invalid.hostname.foo.bar:9999");
props.setProperty(ConsumerConfig.METRIC_REPORTER_CLASSES_CONFIG, MockMetricsReporter.class.getName());
final int oldInitCount = MockMetricsReporter.INIT_COUNT.get();
final int oldCloseCount = MockMetricsReporter.CLOSE_COUNT.get();
try {
KafkaConsumer<byte[], byte[]> consumer = new KafkaConsumer<byte[], byte[]>(
props, null, new ByteArrayDeserializer(), new ByteArrayDeserializer());
} catch (KafkaException e) {
Assert.assertEquals(oldInitCount + 1, MockMetricsReporter.INIT_COUNT.get());
Assert.assertEquals(oldCloseCount + 1, MockMetricsReporter.CLOSE_COUNT.get());
Assert.assertEquals("Failed to construct kafka consumer", e.getMessage());
return;
}
Assert.fail("should have caught an exception and returned");
}
}

View File

@ -0,0 +1,71 @@
/**
* Licensed to the Apache Software Foundation (ASF) under one or more
* contributor license agreements. See the NOTICE file distributed with
* this work for additional information regarding copyright ownership.
* The ASF licenses this file to You under the Apache License, Version 2.0
* (the "License"); you may not use this file except in compliance with
* the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.kafka.clients.producer;
import org.apache.kafka.common.KafkaException;
import org.apache.kafka.common.serialization.ByteArraySerializer;
import org.apache.kafka.test.MockMetricsReporter;
import org.apache.kafka.test.MockSerializer;
import org.junit.Assert;
import org.junit.Test;
import java.util.Properties;
public class KafkaProducerTest {
@Test
public void testConstructorFailureCloseResource() {
Properties props = new Properties();
props.setProperty(ProducerConfig.CLIENT_ID_CONFIG, "testConstructorClose");
props.setProperty(ProducerConfig.BOOTSTRAP_SERVERS_CONFIG, "some.invalid.hostname.foo.bar:9999");
props.setProperty(ProducerConfig.METRIC_REPORTER_CLASSES_CONFIG, MockMetricsReporter.class.getName());
final int oldInitCount = MockMetricsReporter.INIT_COUNT.get();
final int oldCloseCount = MockMetricsReporter.CLOSE_COUNT.get();
try {
KafkaProducer<byte[], byte[]> producer = new KafkaProducer<byte[], byte[]>(
props, new ByteArraySerializer(), new ByteArraySerializer());
} catch (KafkaException e) {
Assert.assertEquals(oldInitCount + 1, MockMetricsReporter.INIT_COUNT.get());
Assert.assertEquals(oldCloseCount + 1, MockMetricsReporter.CLOSE_COUNT.get());
Assert.assertEquals("Failed to construct kafka producer", e.getMessage());
return;
}
Assert.fail("should have caught an exception and returned");
}
@Test
public void testSerializerClose() {
Properties props = new Properties();
props.setProperty(ProducerConfig.CLIENT_ID_CONFIG, "testConstructorClose");
props.setProperty(ProducerConfig.BOOTSTRAP_SERVERS_CONFIG, "localhost:9999");
props.setProperty(ProducerConfig.METRIC_REPORTER_CLASSES_CONFIG, MockMetricsReporter.class.getName());
final int oldInitCount = MockSerializer.INIT_COUNT.get();
final int oldCloseCount = MockSerializer.CLOSE_COUNT.get();
KafkaProducer<byte[], byte[]> producer = new KafkaProducer<byte[], byte[]>(
props, new MockSerializer(), new MockSerializer());
Assert.assertEquals(oldInitCount + 2, MockSerializer.INIT_COUNT.get());
Assert.assertEquals(oldCloseCount, MockSerializer.CLOSE_COUNT.get());
producer.close();
Assert.assertEquals(oldInitCount + 2, MockSerializer.INIT_COUNT.get());
Assert.assertEquals(oldCloseCount + 2, MockSerializer.CLOSE_COUNT.get());
}
}

View File

@ -17,14 +17,22 @@
package org.apache.kafka.clients.producer; package org.apache.kafka.clients.producer;
import static java.util.Arrays.asList; import static java.util.Arrays.asList;
import static java.util.Collections.singletonList;
import static org.junit.Assert.assertEquals; import static org.junit.Assert.assertEquals;
import static org.junit.Assert.assertFalse; import static org.junit.Assert.assertFalse;
import static org.junit.Assert.assertTrue; import static org.junit.Assert.assertTrue;
import static org.junit.Assert.fail; import static org.junit.Assert.fail;
import java.util.ArrayList;
import java.util.concurrent.ExecutionException; import java.util.concurrent.ExecutionException;
import java.util.concurrent.Future; import java.util.concurrent.Future;
import org.apache.kafka.clients.producer.internals.DefaultPartitioner;
import org.apache.kafka.common.Cluster;
import org.apache.kafka.common.Node;
import org.apache.kafka.common.PartitionInfo;
import org.apache.kafka.common.serialization.StringSerializer;
import org.apache.kafka.test.MockSerializer;
import org.junit.Test; import org.junit.Test;
public class MockProducerTest { public class MockProducerTest {
@ -34,23 +42,36 @@ public class MockProducerTest {
@Test @Test
@SuppressWarnings("unchecked") @SuppressWarnings("unchecked")
public void testAutoCompleteMock() throws Exception { public void testAutoCompleteMock() throws Exception {
MockProducer producer = new MockProducer(true); MockProducer<byte[], byte[]> producer = new MockProducer<byte[], byte[]>(true, new MockSerializer(), new MockSerializer());
ProducerRecord<byte[], byte[]> record = new ProducerRecord<byte[], byte[]>(topic, "key".getBytes(), "value".getBytes()); ProducerRecord<byte[], byte[]> record = new ProducerRecord<byte[], byte[]>(topic, "key".getBytes(), "value".getBytes());
Future<RecordMetadata> metadata = producer.send(record); Future<RecordMetadata> metadata = producer.send(record);
assertTrue("Send should be immediately complete", metadata.isDone()); assertTrue("Send should be immediately complete", metadata.isDone());
assertFalse("Send should be successful", isError(metadata)); assertFalse("Send should be successful", isError(metadata));
assertEquals("Offset should be 0", 0L, metadata.get().offset()); assertEquals("Offset should be 0", 0L, metadata.get().offset());
assertEquals(topic, metadata.get().topic()); assertEquals(topic, metadata.get().topic());
assertEquals("We should have the record in our history", asList(record), producer.history()); assertEquals("We should have the record in our history", singletonList(record), producer.history());
producer.clear();
assertEquals("Clear should erase our history", 0, producer.history().size());
}
@Test
public void testPartitioner() throws Exception {
PartitionInfo partitionInfo0 = new PartitionInfo(topic, 0, null, null, null);
PartitionInfo partitionInfo1 = new PartitionInfo(topic, 1, null, null, null);
Cluster cluster = new Cluster(new ArrayList<Node>(0), asList(partitionInfo0, partitionInfo1));
MockProducer<String, String> producer = new MockProducer<String, String>(cluster, true, new DefaultPartitioner(), new StringSerializer(), new StringSerializer());
ProducerRecord<String, String> record = new ProducerRecord<String, String>(topic, "key", "value");
Future<RecordMetadata> metadata = producer.send(record);
assertEquals("Partition should be correct", 1, metadata.get().partition());
producer.clear(); producer.clear();
assertEquals("Clear should erase our history", 0, producer.history().size()); assertEquals("Clear should erase our history", 0, producer.history().size());
} }
@Test @Test
public void testManualCompletion() throws Exception { public void testManualCompletion() throws Exception {
MockProducer producer = new MockProducer(false); MockProducer<byte[], byte[]> producer = new MockProducer<byte[], byte[]>(false, new MockSerializer(), new MockSerializer());
ProducerRecord<byte[], byte[]> record1 = new ProducerRecord<byte[], byte[]>("topic", "key1".getBytes(), "value1".getBytes()); ProducerRecord<byte[], byte[]> record1 = new ProducerRecord<byte[], byte[]>(topic, "key1".getBytes(), "value1".getBytes());
ProducerRecord<byte[], byte[]> record2 = new ProducerRecord<byte[], byte[]>("topic", "key2".getBytes(), "value2".getBytes()); ProducerRecord<byte[], byte[]> record2 = new ProducerRecord<byte[], byte[]>(topic, "key2".getBytes(), "value2".getBytes());
Future<RecordMetadata> md1 = producer.send(record1); Future<RecordMetadata> md1 = producer.send(record1);
assertFalse("Send shouldn't have completed", md1.isDone()); assertFalse("Send shouldn't have completed", md1.isDone());
Future<RecordMetadata> md2 = producer.send(record2); Future<RecordMetadata> md2 = producer.send(record2);

View File

@ -3,9 +3,9 @@
* file distributed with this work for additional information regarding copyright ownership. The ASF licenses this file * 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 * 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 * License. You may obtain a copy of the License at
* *
* http://www.apache.org/licenses/LICENSE-2.0 * 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 * 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 * 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. * specific language governing permissions and limitations under the License.
@ -18,15 +18,15 @@ import static org.junit.Assert.assertTrue;
import java.util.List; import java.util.List;
import org.apache.kafka.clients.producer.Partitioner;
import org.apache.kafka.common.Cluster; import org.apache.kafka.common.Cluster;
import org.apache.kafka.common.Node; import org.apache.kafka.common.Node;
import org.apache.kafka.common.PartitionInfo; import org.apache.kafka.common.PartitionInfo;
import org.junit.Test; import org.junit.Test;
public class PartitionerTest { public class DefaultPartitionerTest {
private byte[] keyBytes = "key".getBytes();
private byte[] key = "key".getBytes(); private Partitioner partitioner = new DefaultPartitioner();
private Partitioner partitioner = new Partitioner();
private Node node0 = new Node(0, "localhost", 99); private Node node0 = new Node(0, "localhost", 99);
private Node node1 = new Node(1, "localhost", 100); private Node node1 = new Node(1, "localhost", 100);
private Node node2 = new Node(2, "localhost", 101); private Node node2 = new Node(2, "localhost", 101);
@ -38,15 +38,10 @@ public class PartitionerTest {
new PartitionInfo(topic, 0, node0, nodes, nodes)); new PartitionInfo(topic, 0, node0, nodes, nodes));
private Cluster cluster = new Cluster(asList(node0, node1, node2), partitions); private Cluster cluster = new Cluster(asList(node0, node1, node2), partitions);
@Test
public void testUserSuppliedPartitioning() {
assertEquals("If the user supplies a partition we should use it.", 0, partitioner.partition("test", key, 0, cluster));
}
@Test @Test
public void testKeyPartitionIsStable() { public void testKeyPartitionIsStable() {
int partition = partitioner.partition("test", key, null, cluster); int partition = partitioner.partition("test", null, keyBytes, null, null, cluster);
assertEquals("Same key should yield same partition", partition, partitioner.partition("test", key, null, cluster)); assertEquals("Same key should yield same partition", partition, partitioner.partition("test", null, keyBytes, null, null, cluster));
} }
@Test @Test
@ -56,7 +51,7 @@ public class PartitionerTest {
int countForPart0 = 0; int countForPart0 = 0;
int countForPart2 = 0; int countForPart2 = 0;
for (int i = 1; i <= 100; i++) { for (int i = 1; i <= 100; i++) {
int part = partitioner.partition("test", null, null, cluster); int part = partitioner.partition("test", null, null, null, null, cluster);
assertTrue("We should never choose a leader-less node in round robin", part == 0 || part == 2); assertTrue("We should never choose a leader-less node in round robin", part == 0 || part == 2);
if (part == 0) if (part == 0)
countForPart0++; countForPart0++;

View File

@ -26,7 +26,10 @@ import java.util.LinkedHashMap;
import java.util.List; import java.util.List;
import java.util.Map; import java.util.Map;
import java.util.Set; import java.util.Set;
import java.util.concurrent.atomic.AtomicInteger;
import org.apache.kafka.clients.producer.Callback;
import org.apache.kafka.clients.producer.RecordMetadata;
import org.apache.kafka.common.Cluster; import org.apache.kafka.common.Cluster;
import org.apache.kafka.common.Node; import org.apache.kafka.common.Node;
import org.apache.kafka.common.PartitionInfo; import org.apache.kafka.common.PartitionInfo;
@ -203,6 +206,44 @@ public class RecordAccumulatorTest {
// but have leaders with other sendable data. // but have leaders with other sendable data.
assertTrue("Next check time should be defined by node2, at most linger time", result.nextReadyCheckDelayMs <= lingerMs); assertTrue("Next check time should be defined by node2, at most linger time", result.nextReadyCheckDelayMs <= lingerMs);
} }
@Test
public void testRetryBackoff() throws Exception {
long lingerMs = Long.MAX_VALUE / 4;
long retryBackoffMs = Long.MAX_VALUE / 2;
final RecordAccumulator accum = new RecordAccumulator(1024, 10 * 1024, CompressionType.NONE, lingerMs, retryBackoffMs, false, metrics, time, metricTags);
long now = time.milliseconds();
accum.append(tp1, key, value, null);
RecordAccumulator.ReadyCheckResult result = accum.ready(cluster, now + lingerMs + 1);
assertEquals("Node1 should be ready", Collections.singleton(node1), result.readyNodes);
Map<Integer, List<RecordBatch>> batches = accum.drain(cluster, result.readyNodes, Integer.MAX_VALUE, now + lingerMs + 1);
assertEquals("Node1 should be the only ready node.", 1, batches.size());
assertEquals("Partition 0 should only have one batch drained.", 1, batches.get(0).size());
// Reenqueue the batch
now = time.milliseconds();
accum.reenqueue(batches.get(0).get(0), now);
// Put message for partition 1 into accumulator
accum.append(tp2, key, value, null);
result = accum.ready(cluster, now + lingerMs + 1);
assertEquals("Node1 should be ready", Collections.singleton(node1), result.readyNodes);
// tp1 should backoff while tp2 should not
batches = accum.drain(cluster, result.readyNodes, Integer.MAX_VALUE, now + lingerMs + 1);
assertEquals("Node1 should be the only ready node.", 1, batches.size());
assertEquals("Node1 should only have one batch drained.", 1, batches.get(0).size());
assertEquals("Node1 should only have one batch for partition 1.", tp2, batches.get(0).get(0).topicPartition);
// Partition 0 can be drained after retry backoff
result = accum.ready(cluster, now + retryBackoffMs + 1);
assertEquals("Node1 should be ready", Collections.singleton(node1), result.readyNodes);
batches = accum.drain(cluster, result.readyNodes, Integer.MAX_VALUE, now + retryBackoffMs + 1);
assertEquals("Node1 should be the only ready node.", 1, batches.size());
assertEquals("Node1 should only have one batch drained.", 1, batches.get(0).size());
assertEquals("Node1 should only have one batch for partition 0.", tp1, batches.get(0).get(0).topicPartition);
}
@Test @Test
public void testFlush() throws Exception { public void testFlush() throws Exception {
@ -227,4 +268,27 @@ public class RecordAccumulatorTest {
assertFalse(accum.hasUnsent()); assertFalse(accum.hasUnsent());
} }
@Test
public void testAbortIncompleteBatches() throws Exception {
long lingerMs = Long.MAX_VALUE;
final AtomicInteger numExceptionReceivedInCallback = new AtomicInteger(0);
final RecordAccumulator accum = new RecordAccumulator(4 * 1024, 64 * 1024, CompressionType.NONE, lingerMs, 100L, false, metrics, time, metricTags);
class TestCallback implements Callback {
@Override
public void onCompletion(RecordMetadata metadata, Exception exception) {
assertTrue(exception.getMessage().equals("Producer is closed forcefully."));
numExceptionReceivedInCallback.incrementAndGet();
}
}
for (int i = 0; i < 100; i++)
accum.append(new TopicPartition(topic, i % 3), key, value, new TestCallback());
RecordAccumulator.ReadyCheckResult result = accum.ready(cluster, time.milliseconds());
assertEquals("No nodes should be ready.", 0, result.readyNodes.size());
accum.abortIncompleteBatches();
assertEquals(numExceptionReceivedInCallback.get(), 100);
assertFalse(accum.hasUnsent());
}
} }

View File

@ -12,16 +12,16 @@
*/ */
package org.apache.kafka.common.config; package org.apache.kafka.common.config;
import static org.junit.Assert.fail;
import java.util.Map;
import java.util.Properties;
import org.apache.kafka.common.config.ConfigDef.Importance; import org.apache.kafka.common.config.ConfigDef.Importance;
import org.apache.kafka.common.config.ConfigDef.Type; import org.apache.kafka.common.config.ConfigDef.Type;
import org.apache.kafka.common.metrics.MetricsReporter; import org.apache.kafka.common.metrics.MetricsReporter;
import org.junit.Test; import org.junit.Test;
import java.util.Map;
import java.util.Properties;
import static org.junit.Assert.fail;
public class AbstractConfigTest { public class AbstractConfigTest {
@Test @Test
@ -73,7 +73,7 @@ public class AbstractConfigTest {
METRIC_REPORTER_CLASSES_DOC); METRIC_REPORTER_CLASSES_DOC);
} }
public TestConfig(Map<? extends Object, ? extends Object> props) { public TestConfig(Map<?, ?> props) {
super(CONFIG, props); super(CONFIG, props);
} }
} }

View File

@ -22,13 +22,11 @@ import java.net.InetSocketAddress;
import java.net.ServerSocket; import java.net.ServerSocket;
import java.net.Socket; import java.net.Socket;
import java.nio.ByteBuffer; import java.nio.ByteBuffer;
import java.util.ArrayList; import java.util.*;
import java.util.Collections;
import java.util.LinkedHashMap;
import java.util.List;
import org.apache.kafka.common.metrics.Metrics; import org.apache.kafka.common.metrics.Metrics;
import org.apache.kafka.common.utils.MockTime; import org.apache.kafka.common.utils.MockTime;
import org.apache.kafka.common.utils.Time;
import org.apache.kafka.common.utils.Utils; import org.apache.kafka.common.utils.Utils;
import org.apache.kafka.test.TestUtils; import org.apache.kafka.test.TestUtils;
import org.junit.After; import org.junit.After;
@ -43,13 +41,15 @@ public class SelectorTest {
private static final int BUFFER_SIZE = 4 * 1024; private static final int BUFFER_SIZE = 4 * 1024;
private EchoServer server; private EchoServer server;
private Time time;
private Selectable selector; private Selectable selector;
@Before @Before
public void setup() throws Exception { public void setup() throws Exception {
this.server = new EchoServer(); this.server = new EchoServer();
this.server.start(); this.server.start();
this.selector = new Selector(new Metrics(), new MockTime() , "MetricGroup", new LinkedHashMap<String, String>()); this.time = new MockTime();
this.selector = new Selector(5000, new Metrics(), time, "MetricGroup", new LinkedHashMap<String, String>());
} }
@After @After
@ -63,7 +63,7 @@ public class SelectorTest {
*/ */
@Test @Test
public void testServerDisconnect() throws Exception { public void testServerDisconnect() throws Exception {
int node = 0; String node = "0";
// connect and do a simple request // connect and do a simple request
blockingConnect(node); blockingConnect(node);
@ -84,7 +84,7 @@ public class SelectorTest {
*/ */
@Test @Test
public void testClientDisconnect() throws Exception { public void testClientDisconnect() throws Exception {
int node = 0; String node = "0";
blockingConnect(node); blockingConnect(node);
selector.disconnect(node); selector.disconnect(node);
selector.send(createSend(node, "hello1")); selector.send(createSend(node, "hello1"));
@ -101,7 +101,7 @@ public class SelectorTest {
*/ */
@Test(expected = IllegalStateException.class) @Test(expected = IllegalStateException.class)
public void testCantSendWithInProgress() throws Exception { public void testCantSendWithInProgress() throws Exception {
int node = 0; String node = "0";
blockingConnect(node); blockingConnect(node);
selector.send(createSend(node, "test1")); selector.send(createSend(node, "test1"));
selector.send(createSend(node, "test2")); selector.send(createSend(node, "test2"));
@ -113,7 +113,7 @@ public class SelectorTest {
*/ */
@Test(expected = IllegalStateException.class) @Test(expected = IllegalStateException.class)
public void testCantSendWithoutConnecting() throws Exception { public void testCantSendWithoutConnecting() throws Exception {
selector.send(createSend(0, "test")); selector.send(createSend("0", "test"));
selector.poll(1000L); selector.poll(1000L);
} }
@ -122,7 +122,7 @@ public class SelectorTest {
*/ */
@Test(expected = IOException.class) @Test(expected = IOException.class)
public void testNoRouteToHost() throws Exception { public void testNoRouteToHost() throws Exception {
selector.connect(0, new InetSocketAddress("asdf.asdf.dsc", server.port), BUFFER_SIZE, BUFFER_SIZE); selector.connect("0", new InetSocketAddress("asdf.asdf.dsc", server.port), BUFFER_SIZE, BUFFER_SIZE);
} }
/** /**
@ -130,7 +130,7 @@ public class SelectorTest {
*/ */
@Test @Test
public void testConnectionRefused() throws Exception { public void testConnectionRefused() throws Exception {
int node = 0; String node = "0";
ServerSocket nonListeningSocket = new ServerSocket(0); ServerSocket nonListeningSocket = new ServerSocket(0);
int nonListeningPort = nonListeningSocket.getLocalPort(); int nonListeningPort = nonListeningSocket.getLocalPort();
selector.connect(node, new InetSocketAddress("localhost", nonListeningPort), BUFFER_SIZE, BUFFER_SIZE); selector.connect(node, new InetSocketAddress("localhost", nonListeningPort), BUFFER_SIZE, BUFFER_SIZE);
@ -151,14 +151,15 @@ public class SelectorTest {
// create connections // create connections
InetSocketAddress addr = new InetSocketAddress("localhost", server.port); InetSocketAddress addr = new InetSocketAddress("localhost", server.port);
for (int i = 0; i < conns; i++) for (int i = 0; i < conns; i++)
selector.connect(i, addr, BUFFER_SIZE, BUFFER_SIZE); selector.connect(Integer.toString(i), addr, BUFFER_SIZE, BUFFER_SIZE);
// send echo requests and receive responses // send echo requests and receive responses
int[] requests = new int[conns]; Map<String, Integer> requests = new HashMap<String, Integer>();
int[] responses = new int[conns]; Map<String, Integer> responses = new HashMap<String, Integer>();
int responseCount = 0; int responseCount = 0;
for (int i = 0; i < conns; i++) for (int i = 0; i < conns; i++) {
selector.send(createSend(i, i + "-" + 0)); String node = Integer.toString(i);
selector.send(createSend(node, node + "-0"));
}
// loop until we complete all requests // loop until we complete all requests
while (responseCount < conns * reqs) { while (responseCount < conns * reqs) {
@ -171,19 +172,27 @@ public class SelectorTest {
for (NetworkReceive receive : selector.completedReceives()) { for (NetworkReceive receive : selector.completedReceives()) {
String[] pieces = asString(receive).split("-"); String[] pieces = asString(receive).split("-");
assertEquals("Should be in the form 'conn-counter'", 2, pieces.length); assertEquals("Should be in the form 'conn-counter'", 2, pieces.length);
assertEquals("Check the source", receive.source(), Integer.parseInt(pieces[0])); assertEquals("Check the source", receive.source(), pieces[0]);
assertEquals("Check that the receive has kindly been rewound", 0, receive.payload().position()); assertEquals("Check that the receive has kindly been rewound", 0, receive.payload().position());
assertEquals("Check the request counter", responses[receive.source()], Integer.parseInt(pieces[1])); if (responses.containsKey(receive.source())) {
responses[receive.source()]++; // increment the expected counter assertEquals("Check the request counter", (int) responses.get(receive.source()), Integer.parseInt(pieces[1]));
responses.put(receive.source(), responses.get(receive.source()) + 1);
} else {
assertEquals("Check the request counter", 0, Integer.parseInt(pieces[1]));
responses.put(receive.source(), 1);
}
responseCount++; responseCount++;
} }
// prepare new sends for the next round // prepare new sends for the next round
for (NetworkSend send : selector.completedSends()) { for (Send send : selector.completedSends()) {
int dest = send.destination(); String dest = send.destination();
requests[dest]++; if (requests.containsKey(dest))
if (requests[dest] < reqs) requests.put(dest, requests.get(dest) + 1);
selector.send(createSend(dest, dest + "-" + requests[dest])); else
requests.put(dest, 1);
if (requests.get(dest) < reqs)
selector.send(createSend(dest, dest + "-" + requests.get(dest)));
} }
} }
} }
@ -193,7 +202,7 @@ public class SelectorTest {
*/ */
@Test @Test
public void testSendLargeRequest() throws Exception { public void testSendLargeRequest() throws Exception {
int node = 0; String node = "0";
blockingConnect(node); blockingConnect(node);
String big = TestUtils.randomString(10 * BUFFER_SIZE); String big = TestUtils.randomString(10 * BUFFER_SIZE);
assertEquals(big, blockingRequest(node, big)); assertEquals(big, blockingRequest(node, big));
@ -204,41 +213,53 @@ public class SelectorTest {
*/ */
@Test @Test
public void testEmptyRequest() throws Exception { public void testEmptyRequest() throws Exception {
int node = 0; String node = "0";
blockingConnect(node); blockingConnect(node);
assertEquals("", blockingRequest(node, "")); assertEquals("", blockingRequest(node, ""));
} }
@Test(expected = IllegalStateException.class) @Test(expected = IllegalStateException.class)
public void testExistingConnectionId() throws IOException { public void testExistingConnectionId() throws IOException {
blockingConnect(0); blockingConnect("0");
blockingConnect(0); blockingConnect("0");
} }
@Test @Test
public void testMute() throws Exception { public void testMute() throws Exception {
blockingConnect(0); blockingConnect("0");
blockingConnect(1); blockingConnect("1");
selector.send(createSend(0, "hello")); selector.send(createSend("0", "hello"));
selector.send(createSend(1, "hi")); selector.send(createSend("1", "hi"));
selector.mute(1); selector.mute("1");
while (selector.completedReceives().isEmpty()) while (selector.completedReceives().isEmpty())
selector.poll(5); selector.poll(5);
assertEquals("We should have only one response", 1, selector.completedReceives().size()); assertEquals("We should have only one response", 1, selector.completedReceives().size());
assertEquals("The response should not be from the muted node", 0, selector.completedReceives().get(0).source()); assertEquals("The response should not be from the muted node", "0", selector.completedReceives().get(0).source());
selector.unmute(1); selector.unmute("1");
do { do {
selector.poll(5); selector.poll(5);
} while (selector.completedReceives().isEmpty()); } while (selector.completedReceives().isEmpty());
assertEquals("We should have only one response", 1, selector.completedReceives().size()); assertEquals("We should have only one response", 1, selector.completedReceives().size());
assertEquals("The response should be from the previously muted node", 1, selector.completedReceives().get(0).source()); assertEquals("The response should be from the previously muted node", "1", selector.completedReceives().get(0).source());
} }
private String blockingRequest(int node, String s) throws IOException {
@Test
public void testCloseOldestConnection() throws Exception {
String id = "0";
blockingConnect(id);
time.sleep(6000); // The max idle time is 5000ms
selector.poll(0);
assertTrue("The idle connection should have been closed", selector.disconnected().contains(id));
}
private String blockingRequest(String node, String s) throws IOException {
selector.send(createSend(node, s)); selector.send(createSend(node, s));
selector.poll(1000L); selector.poll(1000L);
while (true) { while (true) {
@ -250,13 +271,13 @@ public class SelectorTest {
} }
/* connect and wait for the connection to complete */ /* connect and wait for the connection to complete */
private void blockingConnect(int node) throws IOException { private void blockingConnect(String node) throws IOException {
selector.connect(node, new InetSocketAddress("localhost", server.port), BUFFER_SIZE, BUFFER_SIZE); selector.connect(node, new InetSocketAddress("localhost", server.port), BUFFER_SIZE, BUFFER_SIZE);
while (!selector.connected().contains(node)) while (!selector.connected().contains(node))
selector.poll(10000L); selector.poll(10000L);
} }
private NetworkSend createSend(int node, String s) { private NetworkSend createSend(String node, String s) {
return new NetworkSend(node, ByteBuffer.wrap(s.getBytes())); return new NetworkSend(node, ByteBuffer.wrap(s.getBytes()));
} }

View File

@ -43,8 +43,8 @@ public class ProtocolSerializationTest {
new Field("struct", new Schema(new Field("field", Type.INT32)))); new Field("struct", new Schema(new Field("field", Type.INT32))));
this.struct = new Struct(this.schema).set("int8", (byte) 1) this.struct = new Struct(this.schema).set("int8", (byte) 1)
.set("int16", (short) 1) .set("int16", (short) 1)
.set("int32", (int) 1) .set("int32", 1)
.set("int64", (long) 1) .set("int64", 1L)
.set("string", "1") .set("string", "1")
.set("bytes", "1".getBytes()) .set("bytes", "1".getBytes())
.set("array", new Object[] {1}); .set("array", new Object[] {1});

View File

@ -54,6 +54,26 @@ public class SerializationTest {
} }
} }
@Test
public void testIntegerSerializer() {
Integer[] integers = new Integer[]{
423412424,
-41243432
};
String mytopic = "testTopic";
Serializer<Integer> serializer = new IntegerSerializer();
Deserializer<Integer> deserializer = new IntegerDeserializer();
for (Integer integer : integers) {
assertEquals("Should get the original integer after serialization and deserialization",
integer, deserializer.deserialize(mytopic, serializer.serialize(mytopic, integer)));
}
assertEquals("Should support null in serialization and deserialization",
null, deserializer.deserialize(mytopic, serializer.serialize(mytopic, null)));
}
private SerDeser<String> getStringSerDeser(String encoder) { private SerDeser<String> getStringSerDeser(String encoder) {
Map<String, Object> serializerConfigs = new HashMap<String, Object>(); Map<String, Object> serializerConfigs = new HashMap<String, Object>();
serializerConfigs.put("key.serializer.encoding", encoder); serializerConfigs.put("key.serializer.encoding", encoder);

View File

@ -0,0 +1,53 @@
/**
* 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.test;
import org.apache.kafka.common.metrics.KafkaMetric;
import org.apache.kafka.common.metrics.MetricsReporter;
import java.util.List;
import java.util.Map;
import java.util.concurrent.atomic.AtomicInteger;
public class MockMetricsReporter implements MetricsReporter {
public static final AtomicInteger INIT_COUNT = new AtomicInteger(0);
public static final AtomicInteger CLOSE_COUNT = new AtomicInteger(0);
public MockMetricsReporter() {
}
@Override
public void init(List<KafkaMetric> metrics) {
INIT_COUNT.incrementAndGet();
}
@Override
public void metricChange(KafkaMetric metric) {
}
@Override
public void close() {
CLOSE_COUNT.incrementAndGet();
}
@Override
public void configure(Map<String, ?> configs) {
}
}

View File

@ -20,6 +20,7 @@ import java.util.List;
import org.apache.kafka.common.network.NetworkReceive; import org.apache.kafka.common.network.NetworkReceive;
import org.apache.kafka.common.network.NetworkSend; import org.apache.kafka.common.network.NetworkSend;
import org.apache.kafka.common.network.Selectable; import org.apache.kafka.common.network.Selectable;
import org.apache.kafka.common.network.Send;
import org.apache.kafka.common.utils.Time; import org.apache.kafka.common.utils.Time;
/** /**
@ -28,23 +29,23 @@ import org.apache.kafka.common.utils.Time;
public class MockSelector implements Selectable { public class MockSelector implements Selectable {
private final Time time; private final Time time;
private final List<NetworkSend> initiatedSends = new ArrayList<NetworkSend>(); private final List<Send> initiatedSends = new ArrayList<Send>();
private final List<NetworkSend> completedSends = new ArrayList<NetworkSend>(); private final List<Send> completedSends = new ArrayList<Send>();
private final List<NetworkReceive> completedReceives = new ArrayList<NetworkReceive>(); private final List<NetworkReceive> completedReceives = new ArrayList<NetworkReceive>();
private final List<Integer> disconnected = new ArrayList<Integer>(); private final List<String> disconnected = new ArrayList<String>();
private final List<Integer> connected = new ArrayList<Integer>(); private final List<String> connected = new ArrayList<String>();
public MockSelector(Time time) { public MockSelector(Time time) {
this.time = time; this.time = time;
} }
@Override @Override
public void connect(int id, InetSocketAddress address, int sendBufferSize, int receiveBufferSize) throws IOException { public void connect(String id, InetSocketAddress address, int sendBufferSize, int receiveBufferSize) throws IOException {
this.connected.add(id); this.connected.add(id);
} }
@Override @Override
public void disconnect(int id) { public void disconnect(String id) {
this.disconnected.add(id); this.disconnected.add(id);
} }
@ -64,7 +65,7 @@ public class MockSelector implements Selectable {
} }
@Override @Override
public void send(NetworkSend send) { public void send(Send send) {
this.initiatedSends.add(send); this.initiatedSends.add(send);
} }
@ -76,7 +77,7 @@ public class MockSelector implements Selectable {
} }
@Override @Override
public List<NetworkSend> completedSends() { public List<Send> completedSends() {
return completedSends; return completedSends;
} }
@ -94,21 +95,21 @@ public class MockSelector implements Selectable {
} }
@Override @Override
public List<Integer> disconnected() { public List<String> disconnected() {
return disconnected; return disconnected;
} }
@Override @Override
public List<Integer> connected() { public List<String> connected() {
return connected; return connected;
} }
@Override @Override
public void mute(int id) { public void mute(String id) {
} }
@Override @Override
public void unmute(int id) { public void unmute(String id) {
} }
@Override @Override

View File

@ -14,23 +14,32 @@
* See the License for the specific language governing permissions and * See the License for the specific language governing permissions and
* limitations under the License. * limitations under the License.
*/ */
package org.apache.kafka.test;
package kafka.coordinator import org.apache.kafka.common.serialization.Serializer;
import scala.collection.mutable import java.util.Map;
import java.util.concurrent.atomic.AtomicInteger;
/** public class MockSerializer implements Serializer<byte[]> {
* A bucket of consumers that are scheduled for heartbeat expiration. public static final AtomicInteger INIT_COUNT = new AtomicInteger(0);
* public static final AtomicInteger CLOSE_COUNT = new AtomicInteger(0);
* The motivation behind this is to avoid expensive fine-grained per-consumer
* heartbeat expiration but use coarsen-grained methods that group consumers
* with similar deadline together. This will result in some consumers not
* being expired for heartbeats in time but is tolerable.
*/
class HeartbeatBucket(val startMs: Long, endMs: Long) {
/* The list of consumers that are contained in this bucket */ public MockSerializer() {
val consumerRegistryList = new mutable.HashSet[ConsumerRegistry] INIT_COUNT.incrementAndGet();
}
// TODO @Override
public void configure(Map<String, ?> configs, boolean isKey) {
}
@Override
public byte[] serialize(String topic, byte[] data) {
return data;
}
@Override
public void close() {
CLOSE_COUNT.incrementAndGet();
}
} }

View File

@ -159,7 +159,7 @@ public class KafkaETLContext {
_response = _consumer.fetch(fetchRequest); _response = _consumer.fetch(fetchRequest);
if(_response != null) { if(_response != null) {
_respIterator = new ArrayList<ByteBufferMessageSet>(){{ _respIterator = new ArrayList<ByteBufferMessageSet>(){{
add((ByteBufferMessageSet) _response.messageSet(_request.getTopic(), _request.getPartition())); add(_response.messageSet(_request.getTopic(), _request.getPartition()));
}}.iterator(); }}.iterator();
} }
_requestTime += (System.currentTimeMillis() - tempTime); _requestTime += (System.currentTimeMillis() - tempTime);

View File

@ -17,6 +17,8 @@
package kafka package kafka
import java.util.Properties
import scala.collection.JavaConversions._ import scala.collection.JavaConversions._
import joptsimple.OptionParser import joptsimple.OptionParser
import metrics.KafkaMetricsReporter import metrics.KafkaMetricsReporter
@ -26,7 +28,7 @@ import org.apache.kafka.common.utils.Utils
object Kafka extends Logging { object Kafka extends Logging {
def getKafkaConfigFromArgs(args: Array[String]): KafkaConfig = { def getPropsFromArgs(args: Array[String]): Properties = {
val optionParser = new OptionParser val optionParser = new OptionParser
val overrideOpt = optionParser.accepts("override", "Optional property that should override values set in server.properties file") val overrideOpt = optionParser.accepts("override", "Optional property that should override values set in server.properties file")
.withRequiredArg() .withRequiredArg()
@ -47,14 +49,14 @@ object Kafka extends Logging {
props.putAll(CommandLineUtils.parseKeyValueArgs(options.valuesOf(overrideOpt))) props.putAll(CommandLineUtils.parseKeyValueArgs(options.valuesOf(overrideOpt)))
} }
props
KafkaConfig.fromProps(props)
} }
def main(args: Array[String]): Unit = { def main(args: Array[String]): Unit = {
try { try {
val serverConfig = getKafkaConfigFromArgs(args) val serverProps = getPropsFromArgs(args)
KafkaMetricsReporter.startReporters(new VerifiableProperties(serverConfig.toProps)) val serverConfig = KafkaConfig.fromProps(serverProps)
KafkaMetricsReporter.startReporters(new VerifiableProperties(serverProps))
val kafkaServerStartable = new KafkaServerStartable(serverConfig) val kafkaServerStartable = new KafkaServerStartable(serverConfig)
// attach shutdown handler to catch control-c // attach shutdown handler to catch control-c
@ -68,7 +70,9 @@ object Kafka extends Logging {
kafkaServerStartable.awaitShutdown kafkaServerStartable.awaitShutdown
} }
catch { catch {
case e: Throwable => fatal(e) case e: Throwable =>
fatal(e)
System.exit(1)
} }
System.exit(0) System.exit(0)
} }

View File

@ -317,12 +317,17 @@ object AdminUtils extends Logging {
if(str != null) { if(str != null) {
Json.parseFull(str) match { Json.parseFull(str) match {
case None => // there are no config overrides case None => // there are no config overrides
case Some(map: Map[String, _]) => case Some(mapAnon: Map[_, _]) =>
val map = mapAnon collect { case (k: String, v: Any) => k -> v }
require(map("version") == 1) require(map("version") == 1)
map.get("config") match { map.get("config") match {
case Some(config: Map[String, String]) => case Some(config: Map[_, _]) =>
for((k,v) <- config) for(configTup <- config)
props.setProperty(k, v) configTup match {
case (k: String, v: String) =>
props.setProperty(k, v)
case _ => throw new IllegalArgumentException("Invalid topic config: " + str)
}
case _ => throw new IllegalArgumentException("Invalid topic config: " + str) case _ => throw new IllegalArgumentException("Invalid topic config: " + str)
} }

View File

@ -48,7 +48,7 @@ object ConsumerGroupCommand {
opts.checkArgs() opts.checkArgs()
val zkClient = new ZkClient(opts.options.valueOf(opts.zkConnectOpt), 30000, 30000, ZKStringSerializer) val zkClient = ZkUtils.createZkClient(opts.options.valueOf(opts.zkConnectOpt), 30000, 30000)
try { try {
if (opts.options.has(opts.listOpt)) if (opts.options.has(opts.listOpt))
@ -174,7 +174,7 @@ object ConsumerGroupCommand {
val offsetMap = mutable.Map[TopicAndPartition, Long]() val offsetMap = mutable.Map[TopicAndPartition, Long]()
val channel = ClientUtils.channelToOffsetManager(group, zkClient, channelSocketTimeoutMs, channelRetryBackoffMs) val channel = ClientUtils.channelToOffsetManager(group, zkClient, channelSocketTimeoutMs, channelRetryBackoffMs)
channel.send(OffsetFetchRequest(group, topicPartitions)) channel.send(OffsetFetchRequest(group, topicPartitions))
val offsetFetchResponse = OffsetFetchResponse.readFrom(channel.receive().buffer) val offsetFetchResponse = OffsetFetchResponse.readFrom(channel.receive().payload())
offsetFetchResponse.requestInfo.foreach { case (topicAndPartition, offsetAndMetadata) => offsetFetchResponse.requestInfo.foreach { case (topicAndPartition, offsetAndMetadata) =>
if (offsetAndMetadata == OffsetMetadataAndError.NoOffset) { if (offsetAndMetadata == OffsetMetadataAndError.NoOffset) {

View File

@ -53,7 +53,7 @@ object PreferredReplicaLeaderElectionCommand extends Logging {
var zkClient: ZkClient = null var zkClient: ZkClient = null
try { try {
zkClient = new ZkClient(zkConnect, 30000, 30000, ZKStringSerializer) zkClient = ZkUtils.createZkClient(zkConnect, 30000, 30000)
val partitionsForPreferredReplicaElection = val partitionsForPreferredReplicaElection =
if (!options.has(jsonFileOpt)) if (!options.has(jsonFileOpt))
ZkUtils.getAllPartitions(zkClient) ZkUtils.getAllPartitions(zkClient)

View File

@ -38,7 +38,7 @@ object ReassignPartitionsCommand extends Logging {
CommandLineUtils.checkRequiredArgs(opts.parser, opts.options, opts.zkConnectOpt) CommandLineUtils.checkRequiredArgs(opts.parser, opts.options, opts.zkConnectOpt)
val zkConnect = opts.options.valueOf(opts.zkConnectOpt) val zkConnect = opts.options.valueOf(opts.zkConnectOpt)
var zkClient: ZkClient = new ZkClient(zkConnect, 30000, 30000, ZKStringSerializer) var zkClient: ZkClient = ZkUtils.createZkClient(zkConnect, 30000, 30000)
try { try {
if(opts.options.has(opts.verifyOpt)) if(opts.options.has(opts.verifyOpt))
verifyAssignment(zkClient, opts) verifyAssignment(zkClient, opts)
@ -127,7 +127,7 @@ object ReassignPartitionsCommand extends Logging {
} }
val reassignPartitionsCommand = new ReassignPartitionsCommand(zkClient, partitionsToBeReassigned.toMap) val reassignPartitionsCommand = new ReassignPartitionsCommand(zkClient, partitionsToBeReassigned.toMap)
// before starting assignment, output the current replica assignment to facilitate rollback // before starting assignment, output the current replica assignment to facilitate rollback
val currentPartitionReplicaAssignment = ZkUtils.getReplicaAssignmentForTopics(zkClient, partitionsToBeReassigned.map(_._1.topic).toSeq) val currentPartitionReplicaAssignment = ZkUtils.getReplicaAssignmentForTopics(zkClient, partitionsToBeReassigned.map(_._1.topic))
println("Current partition replica assignment\n\n%s\n\nSave this to use as the --reassignment-json-file option during rollback" println("Current partition replica assignment\n\n%s\n\nSave this to use as the --reassignment-json-file option during rollback"
.format(ZkUtils.getPartitionReassignmentZkData(currentPartitionReplicaAssignment))) .format(ZkUtils.getPartitionReassignmentZkData(currentPartitionReplicaAssignment)))
// start the reassignment // start the reassignment

View File

@ -47,7 +47,7 @@ object TopicCommand {
opts.checkArgs() opts.checkArgs()
val zkClient = new ZkClient(opts.options.valueOf(opts.zkConnectOpt), 30000, 30000, ZKStringSerializer) val zkClient = ZkUtils.createZkClient(opts.options.valueOf(opts.zkConnectOpt), 30000, 30000)
try { try {
if(opts.options.has(opts.createOpt)) if(opts.options.has(opts.createOpt))
@ -143,7 +143,7 @@ object TopicCommand {
topics.foreach { topic => topics.foreach { topic =>
try { try {
if (Topic.InternalTopics.contains(topic)) { if (Topic.InternalTopics.contains(topic)) {
throw new AdminOperationException("Topic %s is a kafka internal topic and is not allowed to be marked for deletion.".format(topic)); throw new AdminOperationException("Topic %s is a kafka internal topic and is not allowed to be marked for deletion.".format(topic))
} else { } else {
ZkUtils.createPersistentPath(zkClient, ZkUtils.getDeleteTopicPath(topic)) ZkUtils.createPersistentPath(zkClient, ZkUtils.getDeleteTopicPath(topic))
println("Topic %s is marked for deletion.".format(topic)) println("Topic %s is marked for deletion.".format(topic))

View File

@ -18,9 +18,10 @@
package kafka.api package kafka.api
import java.nio.ByteBuffer import java.nio.ByteBuffer
import kafka.network.{BoundedByteBufferSend, RequestChannel}
import kafka.network.RequestChannel.Response
import kafka.common.ErrorMapping import kafka.common.ErrorMapping
import kafka.network.{RequestOrResponseSend, RequestChannel}
import kafka.network.RequestChannel.Response
object ConsumerMetadataRequest { object ConsumerMetadataRequest {
val CurrentVersion = 0.shortValue val CurrentVersion = 0.shortValue
@ -64,7 +65,7 @@ case class ConsumerMetadataRequest(group: String,
override def handleError(e: Throwable, requestChannel: RequestChannel, request: RequestChannel.Request): Unit = { override def handleError(e: Throwable, requestChannel: RequestChannel, request: RequestChannel.Request): Unit = {
// return ConsumerCoordinatorNotAvailable for all uncaught errors // return ConsumerCoordinatorNotAvailable for all uncaught errors
val errorResponse = ConsumerMetadataResponse(None, ErrorMapping.ConsumerCoordinatorNotAvailableCode, correlationId) val errorResponse = ConsumerMetadataResponse(None, ErrorMapping.ConsumerCoordinatorNotAvailableCode, correlationId)
requestChannel.sendResponse(new Response(request, new BoundedByteBufferSend(errorResponse))) requestChannel.sendResponse(new Response(request, new RequestOrResponseSend(request.connectionId, errorResponse)))
} }
def describe(details: Boolean) = { def describe(details: Boolean) = {

View File

@ -18,10 +18,9 @@
package kafka.api package kafka.api
import java.nio.ByteBuffer import java.nio.ByteBuffer
import kafka.api.ApiUtils._
import collection.mutable.ListBuffer import kafka.common.{ErrorMapping, TopicAndPartition}
import kafka.network.{BoundedByteBufferSend, RequestChannel} import kafka.network.{RequestOrResponseSend, RequestChannel}
import kafka.common.{TopicAndPartition, ErrorMapping}
import kafka.network.RequestChannel.Response import kafka.network.RequestChannel.Response
import kafka.utils.Logging import kafka.utils.Logging
@ -37,9 +36,9 @@ object ControlledShutdownRequest extends Logging {
} }
} }
case class ControlledShutdownRequest(val versionId: Short, case class ControlledShutdownRequest(versionId: Short,
val correlationId: Int, correlationId: Int,
val brokerId: Int) brokerId: Int)
extends RequestOrResponse(Some(RequestKeys.ControlledShutdownKey)){ extends RequestOrResponse(Some(RequestKeys.ControlledShutdownKey)){
def this(correlationId: Int, brokerId: Int) = def this(correlationId: Int, brokerId: Int) =
@ -63,7 +62,7 @@ case class ControlledShutdownRequest(val versionId: Short,
override def handleError(e: Throwable, requestChannel: RequestChannel, request: RequestChannel.Request): Unit = { override def handleError(e: Throwable, requestChannel: RequestChannel, request: RequestChannel.Request): Unit = {
val errorResponse = ControlledShutdownResponse(correlationId, ErrorMapping.codeFor(e.getCause.asInstanceOf[Class[Throwable]]), Set.empty[TopicAndPartition]) val errorResponse = ControlledShutdownResponse(correlationId, ErrorMapping.codeFor(e.getCause.asInstanceOf[Class[Throwable]]), Set.empty[TopicAndPartition])
requestChannel.sendResponse(new Response(request, new BoundedByteBufferSend(errorResponse))) requestChannel.sendResponse(new Response(request, new RequestOrResponseSend(request.connectionId, errorResponse)))
} }
override def describe(details: Boolean = false): String = { override def describe(details: Boolean = false): String = {
@ -74,4 +73,4 @@ case class ControlledShutdownRequest(val versionId: Short,
controlledShutdownRequest.append("; BrokerId: " + brokerId) controlledShutdownRequest.append("; BrokerId: " + brokerId)
controlledShutdownRequest.toString() controlledShutdownRequest.toString()
} }
} }

View File

@ -39,9 +39,9 @@ object ControlledShutdownResponse {
} }
case class ControlledShutdownResponse(val correlationId: Int, case class ControlledShutdownResponse(correlationId: Int,
val errorCode: Short = ErrorMapping.NoError, errorCode: Short = ErrorMapping.NoError,
val partitionsRemaining: Set[TopicAndPartition]) partitionsRemaining: Set[TopicAndPartition])
extends RequestOrResponse() { extends RequestOrResponse() {
def sizeInBytes(): Int ={ def sizeInBytes(): Int ={
var size = var size =
@ -68,4 +68,4 @@ case class ControlledShutdownResponse(val correlationId: Int,
override def describe(details: Boolean):String = { toString } override def describe(details: Boolean):String = { toString }
} }

View File

@ -149,7 +149,7 @@ case class FetchRequest(versionId: Short = FetchRequest.CurrentVersion,
(topicAndPartition, FetchResponsePartitionData(ErrorMapping.codeFor(e.getClass.asInstanceOf[Class[Throwable]]), -1, MessageSet.Empty)) (topicAndPartition, FetchResponsePartitionData(ErrorMapping.codeFor(e.getClass.asInstanceOf[Class[Throwable]]), -1, MessageSet.Empty))
} }
val errorResponse = FetchResponse(correlationId, fetchResponsePartitionData) val errorResponse = FetchResponse(correlationId, fetchResponsePartitionData)
requestChannel.sendResponse(new RequestChannel.Response(request, new FetchResponseSend(errorResponse))) requestChannel.sendResponse(new RequestChannel.Response(request, new FetchResponseSend(request.connectionId, errorResponse)))
} }
override def describe(details: Boolean): String = { override def describe(details: Boolean): String = {

View File

@ -22,8 +22,10 @@ import java.nio.channels.GatheringByteChannel
import kafka.common.{TopicAndPartition, ErrorMapping} import kafka.common.{TopicAndPartition, ErrorMapping}
import kafka.message.{MessageSet, ByteBufferMessageSet} import kafka.message.{MessageSet, ByteBufferMessageSet}
import kafka.network.{MultiSend, Send}
import kafka.api.ApiUtils._ import kafka.api.ApiUtils._
import org.apache.kafka.common.KafkaException
import org.apache.kafka.common.network.Send
import org.apache.kafka.common.network.MultiSend
import scala.collection._ import scala.collection._
@ -62,10 +64,12 @@ class PartitionDataSend(val partitionId: Int,
buffer.putInt(partitionData.messages.sizeInBytes) buffer.putInt(partitionData.messages.sizeInBytes)
buffer.rewind() buffer.rewind()
override def complete = !buffer.hasRemaining && messagesSentSize >= messageSize override def completed = !buffer.hasRemaining && messagesSentSize >= messageSize
override def writeTo(channel: GatheringByteChannel): Int = { override def destination: String = ""
var written = 0
override def writeTo(channel: GatheringByteChannel): Long = {
var written = 0L
if(buffer.hasRemaining) if(buffer.hasRemaining)
written += channel.write(buffer) written += channel.write(buffer)
if(!buffer.hasRemaining && messagesSentSize < messageSize) { if(!buffer.hasRemaining && messagesSentSize < messageSize) {
@ -75,6 +79,8 @@ class PartitionDataSend(val partitionId: Int,
} }
written written
} }
override def size = buffer.capacity() + messageSize
} }
object TopicData { object TopicData {
@ -101,29 +107,32 @@ case class TopicData(topic: String, partitionData: Map[Int, FetchResponsePartiti
val headerSize = TopicData.headerSize(topic) val headerSize = TopicData.headerSize(topic)
} }
class TopicDataSend(val topicData: TopicData) extends Send { class TopicDataSend(val dest: String, val topicData: TopicData) extends Send {
private val size = topicData.sizeInBytes
private var sent = 0 private var sent = 0L
override def complete = sent >= size override def completed: Boolean = sent >= size
override def destination: String = dest
override def size = topicData.headerSize + sends.size()
private val buffer = ByteBuffer.allocate(topicData.headerSize) private val buffer = ByteBuffer.allocate(topicData.headerSize)
writeShortString(buffer, topicData.topic) writeShortString(buffer, topicData.topic)
buffer.putInt(topicData.partitionData.size) buffer.putInt(topicData.partitionData.size)
buffer.rewind() buffer.rewind()
val sends = new MultiSend(topicData.partitionData.toList private val sends = new MultiSend(dest,
.map(d => new PartitionDataSend(d._1, d._2))) { JavaConversions.seqAsJavaList(topicData.partitionData.toList.map(d => new PartitionDataSend(d._1, d._2))))
val expectedBytesToWrite = topicData.sizeInBytes - topicData.headerSize
}
def writeTo(channel: GatheringByteChannel): Int = { override def writeTo(channel: GatheringByteChannel): Long = {
expectIncomplete() if (completed)
var written = 0 throw new KafkaException("This operation cannot be completed on a complete request.")
var written = 0L
if(buffer.hasRemaining) if(buffer.hasRemaining)
written += channel.write(buffer) written += channel.write(buffer)
if(!buffer.hasRemaining && !sends.complete) { if(!buffer.hasRemaining && !sends.completed) {
written += sends.writeTo(channel) written += sends.writeTo(channel)
} }
sent += written sent += written
@ -200,34 +209,36 @@ case class FetchResponse(correlationId: Int, data: Map[TopicAndPartition, FetchR
} }
class FetchResponseSend(val fetchResponse: FetchResponse) extends Send { class FetchResponseSend(val dest: String, val fetchResponse: FetchResponse) extends Send {
private val size = fetchResponse.sizeInBytes private val payloadSize = fetchResponse.sizeInBytes
private var sent = 0 private var sent = 0L
private val sendSize = 4 /* for size */ + size override def size = 4 /* for size byte */ + payloadSize
override def complete = sent >= sendSize override def completed = sent >= size
override def destination = dest
private val buffer = ByteBuffer.allocate(4 /* for size */ + FetchResponse.headerSize) private val buffer = ByteBuffer.allocate(4 /* for size */ + FetchResponse.headerSize)
buffer.putInt(size) buffer.putInt(payloadSize)
buffer.putInt(fetchResponse.correlationId) buffer.putInt(fetchResponse.correlationId)
buffer.putInt(fetchResponse.dataGroupedByTopic.size) // topic count buffer.putInt(fetchResponse.dataGroupedByTopic.size) // topic count
buffer.rewind() buffer.rewind()
val sends = new MultiSend(fetchResponse.dataGroupedByTopic.toList.map { private val sends = new MultiSend(dest, JavaConversions.seqAsJavaList(fetchResponse.dataGroupedByTopic.toList.map {
case(topic, data) => new TopicDataSend(TopicData(topic, case(topic, data) => new TopicDataSend(dest, TopicData(topic,
data.map{case(topicAndPartition, message) => (topicAndPartition.partition, message)})) data.map{case(topicAndPartition, message) => (topicAndPartition.partition, message)}))
}) { }))
val expectedBytesToWrite = fetchResponse.sizeInBytes - FetchResponse.headerSize
}
def writeTo(channel: GatheringByteChannel):Int = { override def writeTo(channel: GatheringByteChannel): Long = {
expectIncomplete() if (completed)
var written = 0 throw new KafkaException("This operation cannot be completed on a complete request.")
var written = 0L
if(buffer.hasRemaining) if(buffer.hasRemaining)
written += channel.write(buffer) written += channel.write(buffer)
if(!buffer.hasRemaining && !sends.complete) { if(!buffer.hasRemaining && !sends.completed) {
written += sends.writeTo(channel) written += sends.writeTo(channel)
} }
sent += written sent += written

View File

@ -36,7 +36,7 @@ private[kafka] abstract class GenericRequestAndHeader(val versionId: Short,
2 /* version id */ + 2 /* version id */ +
4 /* correlation id */ + 4 /* correlation id */ +
(2 + clientId.length) /* client id */ + (2 + clientId.length) /* client id */ +
body.sizeOf(); body.sizeOf()
} }
override def toString(): String = { override def toString(): String = {
@ -52,4 +52,4 @@ private[kafka] abstract class GenericRequestAndHeader(val versionId: Short,
strBuffer.append("; Body: " + body.toString) strBuffer.append("; Body: " + body.toString)
strBuffer.toString() strBuffer.toString()
} }
} }

View File

@ -29,7 +29,7 @@ private[kafka] abstract class GenericResponseAndHeader(val correlationId: Int,
def sizeInBytes(): Int = { def sizeInBytes(): Int = {
4 /* correlation id */ + 4 /* correlation id */ +
body.sizeOf(); body.sizeOf()
} }
override def toString(): String = { override def toString(): String = {
@ -43,4 +43,4 @@ private[kafka] abstract class GenericResponseAndHeader(val correlationId: Int,
strBuffer.append("; Body: " + body.toString) strBuffer.append("; Body: " + body.toString)
strBuffer.toString() strBuffer.toString()
} }
} }

View File

@ -19,14 +19,16 @@
package kafka.api package kafka.api
import java.nio._ import java.nio._
import kafka.utils._
import kafka.api.ApiUtils._ import kafka.api.ApiUtils._
import kafka.cluster.BrokerEndPoint import kafka.cluster.BrokerEndPoint
import kafka.controller.LeaderIsrAndControllerEpoch
import kafka.network.{BoundedByteBufferSend, RequestChannel}
import kafka.common.ErrorMapping import kafka.common.ErrorMapping
import kafka.controller.LeaderIsrAndControllerEpoch
import kafka.network.{RequestOrResponseSend, RequestChannel}
import kafka.network.RequestChannel.Response import kafka.network.RequestChannel.Response
import collection.Set import kafka.utils._
import scala.collection.Set
object LeaderAndIsr { object LeaderAndIsr {
@ -59,8 +61,8 @@ object PartitionStateInfo {
} }
} }
case class PartitionStateInfo(val leaderIsrAndControllerEpoch: LeaderIsrAndControllerEpoch, case class PartitionStateInfo(leaderIsrAndControllerEpoch: LeaderIsrAndControllerEpoch,
val allReplicas: Set[Int]) { allReplicas: Set[Int]) {
def replicationFactor = allReplicas.size def replicationFactor = allReplicas.size
def writeTo(buffer: ByteBuffer) { def writeTo(buffer: ByteBuffer) {
@ -184,7 +186,7 @@ case class LeaderAndIsrRequest (versionId: Short,
case (topicAndPartition, partitionAndState) => (topicAndPartition, ErrorMapping.codeFor(e.getClass.asInstanceOf[Class[Throwable]])) case (topicAndPartition, partitionAndState) => (topicAndPartition, ErrorMapping.codeFor(e.getClass.asInstanceOf[Class[Throwable]]))
} }
val errorResponse = LeaderAndIsrResponse(correlationId, responseMap) val errorResponse = LeaderAndIsrResponse(correlationId, responseMap)
requestChannel.sendResponse(new Response(request, new BoundedByteBufferSend(errorResponse))) requestChannel.sendResponse(new Response(request, new RequestOrResponseSend(request.connectionId, errorResponse)))
} }
override def describe(details: Boolean): String = { override def describe(details: Boolean): String = {
@ -200,4 +202,4 @@ case class LeaderAndIsrRequest (versionId: Short,
leaderAndIsrRequest.append(";PartitionState:" + partitionStateInfos.mkString(",")) leaderAndIsrRequest.append(";PartitionState:" + partitionStateInfos.mkString(","))
leaderAndIsrRequest.toString() leaderAndIsrRequest.toString()
} }
} }

View File

@ -18,11 +18,13 @@
package kafka.api package kafka.api
import java.nio.ByteBuffer import java.nio.ByteBuffer
import kafka.api.ApiUtils._ import kafka.api.ApiUtils._
import kafka.utils.{SystemTime, Logging} import kafka.common.{ErrorMapping, OffsetAndMetadata, TopicAndPartition}
import kafka.network.{RequestChannel, BoundedByteBufferSend} import kafka.network.{RequestOrResponseSend, RequestChannel}
import kafka.common.{OffsetMetadata, OffsetAndMetadata, ErrorMapping, TopicAndPartition}
import kafka.network.RequestChannel.Response import kafka.network.RequestChannel.Response
import kafka.utils.Logging
import scala.collection._ import scala.collection._
object OffsetCommitRequest extends Logging { object OffsetCommitRequest extends Logging {
@ -69,7 +71,8 @@ object OffsetCommitRequest extends Logging {
val partitionId = buffer.getInt val partitionId = buffer.getInt
val offset = buffer.getLong val offset = buffer.getLong
val timestamp = { val timestamp = {
if (versionId <= 1) // version 1 specific field
if (versionId == 1)
buffer.getLong buffer.getLong
else else
org.apache.kafka.common.requests.OffsetCommitRequest.DEFAULT_TIMESTAMP org.apache.kafka.common.requests.OffsetCommitRequest.DEFAULT_TIMESTAMP
@ -126,8 +129,8 @@ case class OffsetCommitRequest(groupId: String,
t1._2.foreach( t2 => { t1._2.foreach( t2 => {
buffer.putInt(t2._1.partition) buffer.putInt(t2._1.partition)
buffer.putLong(t2._2.offset) buffer.putLong(t2._2.offset)
// version 0 and 1 specific data // version 1 specific data
if (versionId <= 1) if (versionId == 1)
buffer.putLong(t2._2.commitTimestamp) buffer.putLong(t2._2.commitTimestamp)
writeShortString(buffer, t2._2.metadata) writeShortString(buffer, t2._2.metadata)
}) })
@ -151,7 +154,7 @@ case class OffsetCommitRequest(groupId: String,
innerCount + innerCount +
4 /* partition */ + 4 /* partition */ +
8 /* offset */ + 8 /* offset */ +
(if (versionId <= 1) 8 else 0) /* timestamp */ + (if (versionId == 1) 8 else 0) /* timestamp */ +
shortStringLength(offsetAndMetadata._2.metadata) shortStringLength(offsetAndMetadata._2.metadata)
}) })
}) })
@ -161,7 +164,7 @@ case class OffsetCommitRequest(groupId: String,
val commitStatus = requestInfo.mapValues(_ => errorCode) val commitStatus = requestInfo.mapValues(_ => errorCode)
val commitResponse = OffsetCommitResponse(commitStatus, correlationId) val commitResponse = OffsetCommitResponse(commitStatus, correlationId)
requestChannel.sendResponse(new Response(request, new BoundedByteBufferSend(commitResponse))) requestChannel.sendResponse(new Response(request, new RequestOrResponseSend(request.connectionId, commitResponse)))
} }
override def describe(details: Boolean): String = { override def describe(details: Boolean): String = {

View File

@ -17,19 +17,16 @@
package kafka.api package kafka.api
import kafka.api.ApiUtils._
import kafka.utils.Logging
import kafka.network.{BoundedByteBufferSend, RequestChannel}
import kafka.common._
import kafka.common.TopicAndPartition
import kafka.network.RequestChannel.Response
import scala.Some
import java.nio.ByteBuffer import java.nio.ByteBuffer
import kafka.api.ApiUtils._
import kafka.common.{TopicAndPartition, _}
import kafka.network.{RequestOrResponseSend, RequestChannel}
import kafka.network.RequestChannel.Response
import kafka.utils.Logging
object OffsetFetchRequest extends Logging { object OffsetFetchRequest extends Logging {
val CurrentVersion: Short = 0 val CurrentVersion: Short = 1
val DefaultClientId = "" val DefaultClientId = ""
def readFrom(buffer: ByteBuffer): OffsetFetchRequest = { def readFrom(buffer: ByteBuffer): OffsetFetchRequest = {
@ -99,7 +96,7 @@ case class OffsetFetchRequest(groupId: String,
)) ))
}.toMap }.toMap
val errorResponse = OffsetFetchResponse(requestInfo=responseMap, correlationId=correlationId) val errorResponse = OffsetFetchResponse(requestInfo=responseMap, correlationId=correlationId)
requestChannel.sendResponse(new Response(request, new BoundedByteBufferSend(errorResponse))) requestChannel.sendResponse(new Response(request, new RequestOrResponseSend(request.connectionId, errorResponse)))
} }
override def describe(details: Boolean): String = { override def describe(details: Boolean): String = {

View File

@ -18,9 +18,10 @@
package kafka.api package kafka.api
import java.nio.ByteBuffer import java.nio.ByteBuffer
import kafka.common.{ErrorMapping, TopicAndPartition}
import kafka.api.ApiUtils._ import kafka.api.ApiUtils._
import kafka.network.{BoundedByteBufferSend, RequestChannel} import kafka.common.{ErrorMapping, TopicAndPartition}
import kafka.network.{RequestOrResponseSend, RequestChannel}
import kafka.network.RequestChannel.Response import kafka.network.RequestChannel.Response
@ -117,7 +118,7 @@ case class OffsetRequest(requestInfo: Map[TopicAndPartition, PartitionOffsetRequ
(topicAndPartition, PartitionOffsetsResponse(ErrorMapping.codeFor(e.getClass.asInstanceOf[Class[Throwable]]), null)) (topicAndPartition, PartitionOffsetsResponse(ErrorMapping.codeFor(e.getClass.asInstanceOf[Class[Throwable]]), null))
} }
val errorResponse = OffsetResponse(correlationId, partitionOffsetResponseMap) val errorResponse = OffsetResponse(correlationId, partitionOffsetResponseMap)
requestChannel.sendResponse(new Response(request, new BoundedByteBufferSend(errorResponse))) requestChannel.sendResponse(new Response(request, new RequestOrResponseSend(request.connectionId, errorResponse)))
} }
override def describe(details: Boolean): String = { override def describe(details: Boolean): String = {

View File

@ -18,11 +18,12 @@
package kafka.api package kafka.api
import java.nio._ import java.nio._
import kafka.message._
import kafka.api.ApiUtils._ import kafka.api.ApiUtils._
import kafka.common._ import kafka.common._
import kafka.message._
import kafka.network.{RequestOrResponseSend, RequestChannel}
import kafka.network.RequestChannel.Response import kafka.network.RequestChannel.Response
import kafka.network.{RequestChannel, BoundedByteBufferSend}
object ProducerRequest { object ProducerRequest {
val CurrentVersion = 0.shortValue val CurrentVersion = 0.shortValue
@ -136,7 +137,7 @@ case class ProducerRequest(versionId: Short = ProducerRequest.CurrentVersion,
(topicAndPartition, ProducerResponseStatus(ErrorMapping.codeFor(e.getClass.asInstanceOf[Class[Throwable]]), -1l)) (topicAndPartition, ProducerResponseStatus(ErrorMapping.codeFor(e.getClass.asInstanceOf[Class[Throwable]]), -1l))
} }
val errorResponse = ProducerResponse(correlationId, producerResponseStatus) val errorResponse = ProducerResponse(correlationId, producerResponseStatus)
requestChannel.sendResponse(new Response(request, new BoundedByteBufferSend(errorResponse))) requestChannel.sendResponse(new Response(request, new RequestOrResponseSend(request.connectionId, errorResponse)))
} }
} }

View File

@ -20,6 +20,8 @@ package kafka.api
import kafka.common.KafkaException import kafka.common.KafkaException
import java.nio.ByteBuffer import java.nio.ByteBuffer
import kafka.network.InvalidRequestException
object RequestKeys { object RequestKeys {
val ProduceKey: Short = 0 val ProduceKey: Short = 0
val FetchKey: Short = 1 val FetchKey: Short = 1
@ -59,7 +61,7 @@ object RequestKeys {
def deserializerForKey(key: Short): (ByteBuffer) => RequestOrResponse = { def deserializerForKey(key: Short): (ByteBuffer) => RequestOrResponse = {
keyToNameAndDeserializerMap.get(key) match { keyToNameAndDeserializerMap.get(key) match {
case Some(nameAndSerializer) => nameAndSerializer._2 case Some(nameAndSerializer) => nameAndSerializer._2
case None => throw new KafkaException("Wrong request type %d".format(key)) case None => throw new InvalidRequestException("Wrong request type %d".format(key))
} }
} }
} }

View File

@ -19,7 +19,7 @@ package kafka.api
import java.nio._ import java.nio._
import kafka.api.ApiUtils._ import kafka.api.ApiUtils._
import kafka.network.{BoundedByteBufferSend, RequestChannel, InvalidRequestException} import kafka.network.{RequestOrResponseSend, RequestChannel, InvalidRequestException}
import kafka.common.{TopicAndPartition, ErrorMapping} import kafka.common.{TopicAndPartition, ErrorMapping}
import kafka.network.RequestChannel.Response import kafka.network.RequestChannel.Response
import kafka.utils.Logging import kafka.utils.Logging
@ -106,7 +106,7 @@ case class StopReplicaRequest(versionId: Short,
case topicAndPartition => (topicAndPartition, ErrorMapping.codeFor(e.getClass.asInstanceOf[Class[Throwable]])) case topicAndPartition => (topicAndPartition, ErrorMapping.codeFor(e.getClass.asInstanceOf[Class[Throwable]]))
}.toMap }.toMap
val errorResponse = StopReplicaResponse(correlationId, responseMap) val errorResponse = StopReplicaResponse(correlationId, responseMap)
requestChannel.sendResponse(new Response(request, new BoundedByteBufferSend(errorResponse))) requestChannel.sendResponse(new Response(request, new RequestOrResponseSend(request.connectionId, errorResponse)))
} }
override def describe(details: Boolean): String = { override def describe(details: Boolean): String = {

View File

@ -42,9 +42,9 @@ object StopReplicaResponse {
} }
case class StopReplicaResponse(val correlationId: Int, case class StopReplicaResponse(correlationId: Int,
val responseMap: Map[TopicAndPartition, Short], responseMap: Map[TopicAndPartition, Short],
val errorCode: Short = ErrorMapping.NoError) errorCode: Short = ErrorMapping.NoError)
extends RequestOrResponse() { extends RequestOrResponse() {
def sizeInBytes(): Int ={ def sizeInBytes(): Int ={
var size = var size =
@ -72,4 +72,4 @@ case class StopReplicaResponse(val correlationId: Int,
} }
override def describe(details: Boolean):String = { toString } override def describe(details: Boolean):String = { toString }
} }

View File

@ -109,7 +109,7 @@ object PartitionMetadata {
} }
case class PartitionMetadata(partitionId: Int, case class PartitionMetadata(partitionId: Int,
val leader: Option[BrokerEndPoint], leader: Option[BrokerEndPoint],
replicas: Seq[BrokerEndPoint], replicas: Seq[BrokerEndPoint],
isr: Seq[BrokerEndPoint] = Seq.empty, isr: Seq[BrokerEndPoint] = Seq.empty,
errorCode: Short = ErrorMapping.NoError) extends Logging { errorCode: Short = ErrorMapping.NoError) extends Logging {

View File

@ -18,13 +18,15 @@
package kafka.api package kafka.api
import java.nio.ByteBuffer import java.nio.ByteBuffer
import kafka.api.ApiUtils._ import kafka.api.ApiUtils._
import collection.mutable.ListBuffer
import kafka.network.{BoundedByteBufferSend, RequestChannel}
import kafka.common.ErrorMapping import kafka.common.ErrorMapping
import kafka.network.{RequestOrResponseSend, RequestChannel}
import kafka.network.RequestChannel.Response import kafka.network.RequestChannel.Response
import kafka.utils.Logging import kafka.utils.Logging
import scala.collection.mutable.ListBuffer
object TopicMetadataRequest extends Logging { object TopicMetadataRequest extends Logging {
val CurrentVersion = 0.shortValue val CurrentVersion = 0.shortValue
val DefaultClientId = "" val DefaultClientId = ""
@ -46,10 +48,10 @@ object TopicMetadataRequest extends Logging {
} }
} }
case class TopicMetadataRequest(val versionId: Short, case class TopicMetadataRequest(versionId: Short,
val correlationId: Int, correlationId: Int,
val clientId: String, clientId: String,
val topics: Seq[String]) topics: Seq[String])
extends RequestOrResponse(Some(RequestKeys.MetadataKey)){ extends RequestOrResponse(Some(RequestKeys.MetadataKey)){
def this(topics: Seq[String], correlationId: Int) = def this(topics: Seq[String], correlationId: Int) =
@ -80,7 +82,7 @@ case class TopicMetadataRequest(val versionId: Short,
topic => TopicMetadata(topic, Nil, ErrorMapping.codeFor(e.getClass.asInstanceOf[Class[Throwable]])) topic => TopicMetadata(topic, Nil, ErrorMapping.codeFor(e.getClass.asInstanceOf[Class[Throwable]]))
} }
val errorResponse = TopicMetadataResponse(Seq(), topicMetadata, correlationId) val errorResponse = TopicMetadataResponse(Seq(), topicMetadata, correlationId)
requestChannel.sendResponse(new Response(request, new BoundedByteBufferSend(errorResponse))) requestChannel.sendResponse(new Response(request, new RequestOrResponseSend(request.connectionId, errorResponse)))
} }
override def describe(details: Boolean): String = { override def describe(details: Boolean): String = {
@ -93,4 +95,4 @@ case class TopicMetadataRequest(val versionId: Short,
topicMetadataRequest.append("; Topics: " + topics.mkString(",")) topicMetadataRequest.append("; Topics: " + topics.mkString(","))
topicMetadataRequest.toString() topicMetadataRequest.toString()
} }
} }

View File

@ -21,8 +21,8 @@ import java.nio.ByteBuffer
import kafka.api.ApiUtils._ import kafka.api.ApiUtils._
import kafka.cluster.{Broker, BrokerEndPoint} import kafka.cluster.{Broker, BrokerEndPoint}
import kafka.common.{ErrorMapping, KafkaException, TopicAndPartition} import kafka.common.{ErrorMapping, KafkaException, TopicAndPartition}
import kafka.network.{RequestOrResponseSend, RequestChannel}
import kafka.network.RequestChannel.Response import kafka.network.RequestChannel.Response
import kafka.network.{BoundedByteBufferSend, RequestChannel}
import org.apache.kafka.common.protocol.SecurityProtocol import org.apache.kafka.common.protocol.SecurityProtocol
import scala.collection.Set import scala.collection.Set
@ -128,7 +128,7 @@ case class UpdateMetadataRequest (versionId: Short,
override def handleError(e: Throwable, requestChannel: RequestChannel, request: RequestChannel.Request): Unit = { override def handleError(e: Throwable, requestChannel: RequestChannel, request: RequestChannel.Request): Unit = {
val errorResponse = new UpdateMetadataResponse(correlationId, ErrorMapping.codeFor(e.getCause.asInstanceOf[Class[Throwable]])) val errorResponse = new UpdateMetadataResponse(correlationId, ErrorMapping.codeFor(e.getCause.asInstanceOf[Class[Throwable]]))
requestChannel.sendResponse(new Response(request, new BoundedByteBufferSend(errorResponse))) requestChannel.sendResponse(new Response(request, new RequestOrResponseSend(request.connectionId, errorResponse)))
} }
override def describe(details: Boolean): String = { override def describe(details: Boolean): String = {

View File

@ -74,7 +74,7 @@ object ClientUtils extends Logging{
} else { } else {
debug("Successfully fetched metadata for %d topic(s) %s".format(topics.size, topics)) debug("Successfully fetched metadata for %d topic(s) %s".format(topics.size, topics))
} }
return topicMetadataResponse topicMetadataResponse
} }
/** /**
@ -153,7 +153,7 @@ object ClientUtils extends Logging{
debug("Querying %s:%d to locate offset manager for %s.".format(queryChannel.host, queryChannel.port, group)) debug("Querying %s:%d to locate offset manager for %s.".format(queryChannel.host, queryChannel.port, group))
queryChannel.send(ConsumerMetadataRequest(group)) queryChannel.send(ConsumerMetadataRequest(group))
val response = queryChannel.receive() val response = queryChannel.receive()
val consumerMetadataResponse = ConsumerMetadataResponse.readFrom(response.buffer) val consumerMetadataResponse = ConsumerMetadataResponse.readFrom(response.payload())
debug("Consumer metadata response: " + consumerMetadataResponse.toString) debug("Consumer metadata response: " + consumerMetadataResponse.toString)
if (consumerMetadataResponse.errorCode == ErrorMapping.NoError) if (consumerMetadataResponse.errorCode == ErrorMapping.NoError)
coordinatorOpt = consumerMetadataResponse.coordinatorOpt coordinatorOpt = consumerMetadataResponse.coordinatorOpt

View File

@ -68,7 +68,7 @@ case class EndPoint(host: String, port: Int, protocolType: SecurityProtocol) {
def writeTo(buffer: ByteBuffer): Unit = { def writeTo(buffer: ByteBuffer): Unit = {
buffer.putInt(port) buffer.putInt(port)
writeShortString(buffer, host) writeShortString(buffer, host)
buffer.putShort(protocolType.id.toShort) buffer.putShort(protocolType.id)
} }
def sizeInBytes: Int = def sizeInBytes: Int =

View File

@ -219,7 +219,7 @@ class Partition(val topic: String,
if (topic == OffsetManager.OffsetsTopicName && if (topic == OffsetManager.OffsetsTopicName &&
/* if we are making a leader->follower transition */ /* if we are making a leader->follower transition */
leaderReplica == localBrokerId) leaderReplica == localBrokerId)
offsetManager.clearOffsetsInPartition(partitionId) offsetManager.removeOffsetsFromCacheForPartition(partitionId)
} }
if (leaderReplicaIdOpt.isDefined && leaderReplicaIdOpt.get == newLeaderBrokerId) { if (leaderReplicaIdOpt.isDefined && leaderReplicaIdOpt.get == newLeaderBrokerId) {

View File

@ -23,4 +23,4 @@ package kafka.common
*/ */
class ConsumerRebalanceFailedException(message: String) extends RuntimeException(message) { class ConsumerRebalanceFailedException(message: String) extends RuntimeException(message) {
def this() = this(null) def this() = this(null)
} }

View File

@ -34,9 +34,9 @@ case class OffsetAndMetadata(offsetMetadata: OffsetMetadata,
commitTimestamp: Long = org.apache.kafka.common.requests.OffsetCommitRequest.DEFAULT_TIMESTAMP, commitTimestamp: Long = org.apache.kafka.common.requests.OffsetCommitRequest.DEFAULT_TIMESTAMP,
expireTimestamp: Long = org.apache.kafka.common.requests.OffsetCommitRequest.DEFAULT_TIMESTAMP) { expireTimestamp: Long = org.apache.kafka.common.requests.OffsetCommitRequest.DEFAULT_TIMESTAMP) {
def offset() = offsetMetadata.offset def offset = offsetMetadata.offset
def metadata() = offsetMetadata.metadata def metadata = offsetMetadata.metadata
override def toString = "[%s,CommitTime %d,ExpirationTime %d]".format(offsetMetadata, commitTimestamp, expireTimestamp) override def toString = "[%s,CommitTime %d,ExpirationTime %d]".format(offsetMetadata, commitTimestamp, expireTimestamp)
} }

View File

@ -104,8 +104,6 @@ class ConsumerConfig private (val props: VerifiableProperties) extends ZKConfig(
/** the socket timeout for network requests. Its value should be at least fetch.wait.max.ms. */ /** the socket timeout for network requests. Its value should be at least fetch.wait.max.ms. */
val socketTimeoutMs = props.getInt("socket.timeout.ms", SocketTimeout) val socketTimeoutMs = props.getInt("socket.timeout.ms", SocketTimeout)
require(fetchWaitMaxMs <= socketTimeoutMs, "socket.timeout.ms should always be at least fetch.wait.max.ms" +
" to prevent unnecessary socket timeouts")
/** the socket receive buffer for network requests */ /** the socket receive buffer for network requests */
val socketReceiveBufferBytes = props.getInt("socket.receive.buffer.bytes", SocketBufferSize) val socketReceiveBufferBytes = props.getInt("socket.receive.buffer.bytes", SocketBufferSize)
@ -133,6 +131,8 @@ class ConsumerConfig private (val props: VerifiableProperties) extends ZKConfig(
/** the maximum amount of time the server will block before answering the fetch request if there isn't sufficient data to immediately satisfy fetch.min.bytes */ /** the maximum amount of time the server will block before answering the fetch request if there isn't sufficient data to immediately satisfy fetch.min.bytes */
val fetchWaitMaxMs = props.getInt("fetch.wait.max.ms", MaxFetchWaitMs) val fetchWaitMaxMs = props.getInt("fetch.wait.max.ms", MaxFetchWaitMs)
require(fetchWaitMaxMs <= socketTimeoutMs, "socket.timeout.ms should always be at least fetch.wait.max.ms" +
" to prevent unnecessary socket timeouts")
/** backoff time between retries during rebalance */ /** backoff time between retries during rebalance */
val rebalanceBackoffMs = props.getInt("rebalance.backoff.ms", zkSyncTimeMs) val rebalanceBackoffMs = props.getInt("rebalance.backoff.ms", zkSyncTimeMs)

View File

@ -104,10 +104,8 @@ class ConsumerIterator[K, V](private val channel: BlockingQueue[FetchedDataChunk
} }
def clearCurrentChunk() { def clearCurrentChunk() {
try { debug("Clearing the current data chunk for this consumer iterator")
debug("Clearing the current data chunk for this consumer iterator") current.set(null)
current.set(null)
}
} }
} }

View File

@ -17,10 +17,14 @@
package kafka.consumer package kafka.consumer
import java.nio.channels.ClosedByInterruptException
import kafka.api._ import kafka.api._
import kafka.network._ import kafka.network._
import kafka.utils._ import kafka.utils._
import kafka.common.{ErrorMapping, TopicAndPartition} import kafka.common.{ErrorMapping, TopicAndPartition}
import org.apache.kafka.common.network.{NetworkReceive, Receive}
import org.apache.kafka.common.utils.Utils._ import org.apache.kafka.common.utils.Utils._
/** /**
@ -62,14 +66,16 @@ class SimpleConsumer(val host: String,
} }
} }
private def sendRequest(request: RequestOrResponse): Receive = { private def sendRequest(request: RequestOrResponse): NetworkReceive = {
lock synchronized { lock synchronized {
var response: Receive = null var response: NetworkReceive = null
try { try {
getOrMakeConnection() getOrMakeConnection()
blockingChannel.send(request) blockingChannel.send(request)
response = blockingChannel.receive() response = blockingChannel.receive()
} catch { } catch {
case e : ClosedByInterruptException =>
throw e
case e : Throwable => case e : Throwable =>
info("Reconnect due to socket error: %s".format(e.toString)) info("Reconnect due to socket error: %s".format(e.toString))
// retry once // retry once
@ -89,12 +95,12 @@ class SimpleConsumer(val host: String,
def send(request: TopicMetadataRequest): TopicMetadataResponse = { def send(request: TopicMetadataRequest): TopicMetadataResponse = {
val response = sendRequest(request) val response = sendRequest(request)
TopicMetadataResponse.readFrom(response.buffer) TopicMetadataResponse.readFrom(response.payload())
} }
def send(request: ConsumerMetadataRequest): ConsumerMetadataResponse = { def send(request: ConsumerMetadataRequest): ConsumerMetadataResponse = {
val response = sendRequest(request) val response = sendRequest(request)
ConsumerMetadataResponse.readFrom(response.buffer) ConsumerMetadataResponse.readFrom(response.payload())
} }
/** /**
@ -104,7 +110,7 @@ class SimpleConsumer(val host: String,
* @return a set of fetched messages * @return a set of fetched messages
*/ */
def fetch(request: FetchRequest): FetchResponse = { def fetch(request: FetchRequest): FetchResponse = {
var response: Receive = null var response: NetworkReceive = null
val specificTimer = fetchRequestAndResponseStats.getFetchRequestAndResponseStats(host, port).requestTimer val specificTimer = fetchRequestAndResponseStats.getFetchRequestAndResponseStats(host, port).requestTimer
val aggregateTimer = fetchRequestAndResponseStats.getFetchRequestAndResponseAllBrokersStats.requestTimer val aggregateTimer = fetchRequestAndResponseStats.getFetchRequestAndResponseAllBrokersStats.requestTimer
aggregateTimer.time { aggregateTimer.time {
@ -112,7 +118,7 @@ class SimpleConsumer(val host: String,
response = sendRequest(request) response = sendRequest(request)
} }
} }
val fetchResponse = FetchResponse.readFrom(response.buffer) val fetchResponse = FetchResponse.readFrom(response.payload())
val fetchedSize = fetchResponse.sizeInBytes val fetchedSize = fetchResponse.sizeInBytes
fetchRequestAndResponseStats.getFetchRequestAndResponseStats(host, port).requestSizeHist.update(fetchedSize) fetchRequestAndResponseStats.getFetchRequestAndResponseStats(host, port).requestSizeHist.update(fetchedSize)
fetchRequestAndResponseStats.getFetchRequestAndResponseAllBrokersStats.requestSizeHist.update(fetchedSize) fetchRequestAndResponseStats.getFetchRequestAndResponseAllBrokersStats.requestSizeHist.update(fetchedSize)
@ -124,7 +130,7 @@ class SimpleConsumer(val host: String,
* @param request a [[kafka.api.OffsetRequest]] object. * @param request a [[kafka.api.OffsetRequest]] object.
* @return a [[kafka.api.OffsetResponse]] object. * @return a [[kafka.api.OffsetResponse]] object.
*/ */
def getOffsetsBefore(request: OffsetRequest) = OffsetResponse.readFrom(sendRequest(request).buffer) def getOffsetsBefore(request: OffsetRequest) = OffsetResponse.readFrom(sendRequest(request).payload())
/** /**
* Commit offsets for a topic * Commit offsets for a topic
@ -135,7 +141,7 @@ class SimpleConsumer(val host: String,
def commitOffsets(request: OffsetCommitRequest) = { def commitOffsets(request: OffsetCommitRequest) = {
// TODO: With KAFKA-1012, we have to first issue a ConsumerMetadataRequest and connect to the coordinator before // TODO: With KAFKA-1012, we have to first issue a ConsumerMetadataRequest and connect to the coordinator before
// we can commit offsets. // we can commit offsets.
OffsetCommitResponse.readFrom(sendRequest(request).buffer) OffsetCommitResponse.readFrom(sendRequest(request).payload())
} }
/** /**
@ -144,7 +150,7 @@ class SimpleConsumer(val host: String,
* @param request a [[kafka.api.OffsetFetchRequest]] object. * @param request a [[kafka.api.OffsetFetchRequest]] object.
* @return a [[kafka.api.OffsetFetchResponse]] object. * @return a [[kafka.api.OffsetFetchResponse]] object.
*/ */
def fetchOffsets(request: OffsetFetchRequest) = OffsetFetchResponse.readFrom(sendRequest(request).buffer) def fetchOffsets(request: OffsetFetchRequest) = OffsetFetchResponse.readFrom(sendRequest(request).payload())
private def getOrMakeConnection() { private def getOrMakeConnection() {
if(!isClosed && !blockingChannel.isConnected) { if(!isClosed && !blockingChannel.isConnected) {

View File

@ -178,7 +178,7 @@ private[kafka] class ZookeeperConsumerConnector(val config: ConsumerConfig,
private def connectZk() { private def connectZk() {
info("Connecting to zookeeper instance at " + config.zkConnect) info("Connecting to zookeeper instance at " + config.zkConnect)
zkClient = new ZkClient(config.zkConnect, config.zkSessionTimeoutMs, config.zkConnectionTimeoutMs, ZKStringSerializer) zkClient = ZkUtils.createZkClient(config.zkConnect, config.zkSessionTimeoutMs, config.zkConnectionTimeoutMs)
} }
// Blocks until the offset manager is located and a channel is established to it. // Blocks until the offset manager is located and a channel is established to it.
@ -334,7 +334,7 @@ private[kafka] class ZookeeperConsumerConnector(val config: ConsumerConfig,
try { try {
kafkaCommitMeter.mark(offsetsToCommit.size) kafkaCommitMeter.mark(offsetsToCommit.size)
offsetsChannel.send(offsetCommitRequest) offsetsChannel.send(offsetCommitRequest)
val offsetCommitResponse = OffsetCommitResponse.readFrom(offsetsChannel.receive().buffer) val offsetCommitResponse = OffsetCommitResponse.readFrom(offsetsChannel.receive().payload())
trace("Offset commit response: %s.".format(offsetCommitResponse)) trace("Offset commit response: %s.".format(offsetCommitResponse))
val (commitFailed, retryableIfFailed, shouldRefreshCoordinator, errorCount) = { val (commitFailed, retryableIfFailed, shouldRefreshCoordinator, errorCount) = {
@ -421,7 +421,7 @@ private[kafka] class ZookeeperConsumerConnector(val config: ConsumerConfig,
ensureOffsetManagerConnected() ensureOffsetManagerConnected()
try { try {
offsetsChannel.send(offsetFetchRequest) offsetsChannel.send(offsetFetchRequest)
val offsetFetchResponse = OffsetFetchResponse.readFrom(offsetsChannel.receive().buffer) val offsetFetchResponse = OffsetFetchResponse.readFrom(offsetsChannel.receive().payload())
trace("Offset fetch response: %s.".format(offsetFetchResponse)) trace("Offset fetch response: %s.".format(offsetFetchResponse))
val (leaderChanged, loadInProgress) = val (leaderChanged, loadInProgress) =
@ -507,6 +507,9 @@ private[kafka] class ZookeeperConsumerConnector(val config: ConsumerConfig,
// The child change watchers will be set inside rebalance when we read the children list. // The child change watchers will be set inside rebalance when we read the children list.
} }
override def handleSessionEstablishmentError(error: Throwable): Unit = {
fatal("Could not establish session with zookeeper", error)
}
} }
class ZKTopicPartitionChangeListener(val loadBalancerListener: ZKRebalancerListener) class ZKTopicPartitionChangeListener(val loadBalancerListener: ZKRebalancerListener)

View File

@ -18,7 +18,7 @@
package kafka.consumer package kafka.consumer
import scala.collection.JavaConversions._ import scala.collection.JavaConversions._
import kafka.utils.{ZkUtils, ZKStringSerializer, Logging} import kafka.utils.{ZkUtils, Logging}
import org.I0Itec.zkclient.{IZkStateListener, IZkChildListener, ZkClient} import org.I0Itec.zkclient.{IZkStateListener, IZkChildListener, ZkClient}
import org.apache.zookeeper.Watcher.Event.KeeperState import org.apache.zookeeper.Watcher.Event.KeeperState
@ -93,6 +93,10 @@ class ZookeeperTopicEventWatcher(val zkClient: ZkClient,
} }
} }
} }
override def handleSessionEstablishmentError(error: Throwable): Unit = {
//no-op ZookeeperConsumerConnector should log error.
}
} }
} }

Some files were not shown because too many files have changed in this diff Show More