Merged in upstream trunk

This commit is contained in:
Geoff Anderson 2015-06-16 13:11:42 -07:00
commit 4034dd6c96
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.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 ###
./gradlew testJar

View File

@ -30,11 +30,14 @@ allprojects {
}
ext {
gradleVersion = "2.0"
}
gradleVersion = "2.4"
def isVerificationRequired(project) {
project.gradle.startParameter.taskNames.any { it.contains("upload") }
skipSigning = project.hasProperty('skipSigning') && skipSigning.toBoolean()
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')
@ -47,32 +50,33 @@ subprojects {
apply plugin: 'maven'
apply plugin: 'signing'
sourceCompatibility = 1.6
licenseTest.onlyIf { isVerificationRequired(project) }
uploadArchives {
repositories {
signing {
if (isVerificationRequired(project)) {
sign configurations.archives
required { shouldSign }
sign configurations.archives
// To test locally, replace mavenUrl in ~/.gradle/gradle.properties to file://localhost/tmp/myRepo/
mavenDeployer {
beforeDeployment { MavenDeployment deployment -> signing.signPom(deployment) }
repository(url: "${mavenUrl}") {
authentication(userName: "${mavenUsername}", password: "${mavenPassword}")
}
afterEvaluate {
pom.artifactId = "${archivesBaseName}"
pom.project {
name 'Apache Kafka'
packaging 'jar'
url 'http://kafka.apache.org'
licenses {
license {
name 'The Apache Software License, Version 2.0'
url 'http://www.apache.org/licenses/LICENSE-2.0.txt'
distribution 'repo'
}
// To test locally, replace mavenUrl in ~/.gradle/gradle.properties to file://localhost/tmp/myRepo/
mavenDeployer {
beforeDeployment { MavenDeployment deployment -> signing.signPom(deployment) }
repository(url: "${mavenUrl}") {
authentication(userName: "${mavenUsername}", password: "${mavenPassword}")
}
afterEvaluate {
pom.artifactId = "${archivesBaseName}"
pom.project {
name 'Apache Kafka'
packaging 'jar'
url 'http://kafka.apache.org'
licenses {
license {
name 'The Apache Software License, Version 2.0'
url 'http://www.apache.org/licenses/LICENSE-2.0.txt'
distribution 'repo'
}
}
}
@ -205,7 +209,7 @@ project(':core') {
compile project(':clients')
compile "org.scala-lang:scala-library:$scalaVersion"
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 'net.sf.jopt-simple:jopt-simple:3.2'
@ -224,7 +228,7 @@ project(':core') {
}
testRuntime "$slf4jlog4j"
zinc 'com.typesafe.zinc:zinc:0.3.1'
zinc 'com.typesafe.zinc:zinc:0.3.7'
}
configurations {
@ -350,7 +354,7 @@ project(':clients') {
dependencies {
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'
testCompile 'com.novocode:junit-interface:0.9'

View File

@ -12,16 +12,21 @@
*/
package org.apache.kafka.clients;
import java.io.Closeable;
import java.net.InetSocketAddress;
import java.util.ArrayList;
import java.util.List;
import java.util.concurrent.atomic.AtomicReference;
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.getPort;
public class ClientUtils {
private static final Logger log = LoggerFactory.getLogger(ClientUtils.class);
public static List<InetSocketAddress> parseAndValidateAddresses(List<String> urls) {
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);
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 {
private final long reconnectBackoffMs;
private final Map<Integer, NodeConnectionState> nodeState;
private final Map<String, NodeConnectionState> nodeState;
public ClusterConnectionStates(long 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.
* @param node The node id to check
* @param id The connection id to check
* @param now The current time in MS
* @return true if we can initiate a new connection
*/
public boolean canConnect(int node, long now) {
NodeConnectionState state = nodeState.get(node);
public boolean canConnect(String id, long now) {
NodeConnectionState state = nodeState.get(id);
if (state == null)
return true;
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
* @param node The node to check
* @param id The connection to check
* @param now The current time in ms
*/
public boolean isBlackedOut(int node, long now) {
NodeConnectionState state = nodeState.get(node);
public boolean isBlackedOut(String id, long now) {
NodeConnectionState state = nodeState.get(id);
if (state == null)
return false;
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
* disconnected, this respects the reconnect backoff time. When connecting or connected, this handles slow/stalled
* connections.
* @param node The node to check
* @param id The connection to check
* @param now The current time in ms
*/
public long connectionDelay(int node, long now) {
NodeConnectionState state = nodeState.get(node);
public long connectionDelay(String id, long now) {
NodeConnectionState state = nodeState.get(id);
if (state == null) return 0;
long timeWaited = now - state.lastConnectAttemptMs;
if (state.state == ConnectionState.DISCONNECTED) {
@ -77,67 +77,67 @@ final class ClusterConnectionStates {
}
/**
* Enter the connecting state for the given node.
* @param node The id of the node we are connecting to
* Enter the connecting state for the given connection.
* @param id The id of the connection
* @param now The current time.
*/
public void connecting(int node, long now) {
nodeState.put(node, new NodeConnectionState(ConnectionState.CONNECTING, now));
public void connecting(String id, long now) {
nodeState.put(id, new NodeConnectionState(ConnectionState.CONNECTING, now));
}
/**
* Return true iff we have a connection to the give node
* @param node The id of the node to check
* Return true iff a specific connection is connected
* @param id The id of the connection to check
*/
public boolean isConnected(int node) {
NodeConnectionState state = nodeState.get(node);
public boolean isConnected(String id) {
NodeConnectionState state = nodeState.get(id);
return state != null && state.state == ConnectionState.CONNECTED;
}
/**
* Return true iff we are in the process of connecting to the given node
* @param node The id of the node
* Return true iff we are in the process of connecting
* @param id The id of the connection
*/
public boolean isConnecting(int node) {
NodeConnectionState state = nodeState.get(node);
public boolean isConnecting(String id) {
NodeConnectionState state = nodeState.get(id);
return state != null && state.state == ConnectionState.CONNECTING;
}
/**
* Enter the connected state for the given node
* @param node The node we have connected to
* Enter the connected state for the given connection
* @param id The connection identifier
*/
public void connected(int node) {
NodeConnectionState nodeState = nodeState(node);
public void connected(String id) {
NodeConnectionState nodeState = nodeState(id);
nodeState.state = ConnectionState.CONNECTED;
}
/**
* 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) {
NodeConnectionState nodeState = nodeState(node);
public void disconnected(String id) {
NodeConnectionState nodeState = nodeState(id);
nodeState.state = ConnectionState.DISCONNECTED;
}
/**
* Get the state of our connection to the given node
* @param node The id of the node
* Get the state of a given connection
* @param id The id of the connection
* @return The state of our connection
*/
public ConnectionState connectionState(int node) {
return nodeState(node).state;
public ConnectionState connectionState(String id) {
return nodeState(id).state;
}
/**
* 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) {
NodeConnectionState state = this.nodeState.get(node);
private NodeConnectionState nodeState(String id) {
NodeConnectionState state = this.nodeState.get(id);
if (state == null)
throw new IllegalStateException("No entry found for node " + node);
throw new IllegalStateException("No entry found for connection " + id);
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_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 {
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) {
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) {
Deque<ClientRequest> reqs = this.requests.get(request.request().destination());
@ -45,7 +45,7 @@ final class InFlightRequests {
/**
* 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);
if (reqs == null || reqs.isEmpty())
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
*/
public ClientRequest completeNext(int node) {
public ClientRequest completeNext(String node) {
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)
* @param node The node id
*/
public ClientRequest lastSent(int node) {
public ClientRequest lastSent(String node) {
return requestQueue(node).peekFirst();
}
@ -72,7 +72,7 @@ final class InFlightRequests {
* @param node The node the request was sent to
* @return The request
*/
public ClientRequest completeLastSent(int node) {
public ClientRequest completeLastSent(String node) {
return requestQueue(node).pollFirst();
}
@ -82,7 +82,7 @@ final class InFlightRequests {
* @param node Node in question
* @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);
return queue == null || queue.isEmpty() ||
(queue.peekFirst().request().completed() && queue.size() < this.maxInFlightRequestsPerConnection);
@ -93,7 +93,7 @@ final class InFlightRequests {
* @param node The node
* @return The request count.
*/
public int inFlightRequestCount(int node) {
public int inFlightRequestCount(String node) {
Deque<ClientRequest> queue = requests.get(node);
return queue == null ? 0 : queue.size();
}
@ -114,7 +114,7 @@ final class InFlightRequests {
* @param node The node
* @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);
if (reqs == null) {
return Collections.emptyList();

View File

@ -12,6 +12,7 @@
*/
package org.apache.kafka.clients;
import java.io.Closeable;
import java.util.List;
import org.apache.kafka.common.Node;
@ -21,7 +22,7 @@ import org.apache.kafka.common.requests.RequestHeader;
/**
* 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
@ -80,13 +81,13 @@ public interface KafkaClient {
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
* @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
@ -116,7 +117,7 @@ public interface KafkaClient {
*
* @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
@ -130,9 +131,4 @@ public interface KafkaClient {
*/
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 int version;
private long lastRefreshMs;
private long lastSuccessfulRefreshMs;
private Cluster cluster;
private boolean needUpdate;
private final Set<String> topics;
@ -57,6 +58,7 @@ public final class Metadata {
this.refreshBackoffMs = refreshBackoffMs;
this.metadataExpireMs = metadataExpireMs;
this.lastRefreshMs = 0L;
this.lastSuccessfulRefreshMs = 0L;
this.version = 0;
this.cluster = Cluster.empty();
this.needUpdate = false;
@ -83,7 +85,7 @@ public final class Metadata {
* is now
*/
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;
return Math.max(timeToExpire, timeToAllowUpdate);
}
@ -146,6 +148,7 @@ public final class Metadata {
public synchronized void update(Cluster cluster, long now) {
this.needUpdate = false;
this.lastRefreshMs = now;
this.lastSuccessfulRefreshMs = now;
this.version += 1;
this.cluster = cluster;
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() {
return this.lastRefreshMs;
public synchronized long lastSuccessfulUpdate() {
return this.lastSuccessfulRefreshMs;
}
/**

View File

@ -22,8 +22,8 @@ import java.util.Set;
import org.apache.kafka.common.Cluster;
import org.apache.kafka.common.Node;
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.Send;
import org.apache.kafka.common.protocol.ApiKeys;
import org.apache.kafka.common.protocol.ProtoUtils;
import org.apache.kafka.common.protocol.types.Struct;
@ -111,7 +111,7 @@ public class NetworkClient implements KafkaClient {
if (isReady(node, now))
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
initiateConnect(node, now);
@ -129,7 +129,7 @@ public class NetworkClient implements KafkaClient {
*/
@Override
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
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
public boolean isReady(Node node, long now) {
int nodeId = node.id();
String nodeId = node.idString();
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
// 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
*/
private boolean isSendable(int node) {
private boolean isSendable(String node) {
return connectionStates.isConnected(node) && inFlightRequests.canSendMore(node);
}
@ -179,7 +179,7 @@ public class NetworkClient implements KafkaClient {
* @param node The node to check
* @return The connection state
*/
public ConnectionState connectionState(int node) {
public ConnectionState connectionState(String node) {
return connectionStates.connectionState(node);
}
@ -190,7 +190,7 @@ public class NetworkClient implements KafkaClient {
*/
@Override
public void send(ClientRequest request) {
int nodeId = request.request().destination();
String nodeId = request.request().destination();
if (!isSendable(nodeId))
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
*/
@Override
public List<ClientResponse> completeAll(int node, long now) {
public List<ClientResponse> completeAll(String node, long now) {
try {
this.selector.muteAll();
this.selector.unmute(node);
@ -288,8 +288,8 @@ public class NetworkClient implements KafkaClient {
* Get the number of in-flight requests for a given node
*/
@Override
public int inFlightRequestCount(int nodeId) {
return this.inFlightRequests.inFlightRequestCount(nodeId);
public int inFlightRequestCount(String node) {
return this.inFlightRequests.inFlightRequestCount(node);
}
/**
@ -334,11 +334,11 @@ public class NetworkClient implements KafkaClient {
for (int i = 0; i < nodes.size(); i++) {
int idx = Utils.abs((this.nodeIndexOffset + i) % nodes.size());
Node node = nodes.get(idx);
int currInflight = this.inFlightRequests.inFlightRequestCount(node.id());
if (currInflight == 0 && this.connectionStates.isConnected(node.id())) {
int currInflight = this.inFlightRequests.inFlightRequestCount(node.idString());
if (currInflight == 0 && this.connectionStates.isConnected(node.idString())) {
// if we find an established connection with no in-flight requests we can stop right away
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
inflight = currInflight;
found = node;
@ -355,7 +355,7 @@ public class NetworkClient implements KafkaClient {
*/
private void handleCompletedSends(List<ClientResponse> responses, long now) {
// 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());
if (!request.expectResponse()) {
this.inFlightRequests.completeLastSent(send.destination());
@ -372,7 +372,7 @@ public class NetworkClient implements KafkaClient {
*/
private void handleCompletedReceives(List<ClientResponse> responses, long now) {
for (NetworkReceive receive : this.selector.completedReceives()) {
int source = receive.source();
String source = receive.source();
ClientRequest req = inFlightRequests.completeNext(source);
ResponseHeader header = ResponseHeader.parse(receive.payload());
short apiKey = req.request().header().apiKey();
@ -391,6 +391,10 @@ public class NetworkClient implements KafkaClient {
this.metadataFetchInProgress = false;
MetadataResponse response = new MetadataResponse(body);
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
// created which means we will get errors and no nodes until it exists
if (cluster.nodes().size() > 0) {
@ -408,7 +412,7 @@ public class NetworkClient implements KafkaClient {
* @param now The current time
*/
private void handleDisconnections(List<ClientResponse> responses, long now) {
for (int node : this.selector.disconnected()) {
for (String node : this.selector.disconnected()) {
connectionStates.disconnected(node);
log.debug("Node {} disconnected.", node);
for (ClientRequest request : this.inFlightRequests.clearAll(node)) {
@ -429,9 +433,9 @@ public class NetworkClient implements KafkaClient {
* Record any newly completed connections
*/
private void handleConnections() {
for (Integer id : this.selector.connected()) {
log.debug("Completed connection to node {}", id);
this.connectionStates.connected(id);
for (String node : this.selector.connected()) {
log.debug("Completed connection to node {}", node);
this.connectionStates.connected(node);
}
}
@ -447,7 +451,7 @@ public class NetworkClient implements KafkaClient {
/**
* 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));
RequestSend send = new RequestSend(node, nextRequestHeader(ApiKeys.METADATA), metadata.toStruct());
return new ClientRequest(now, true, send, null);
@ -466,15 +470,17 @@ public class NetworkClient implements KafkaClient {
this.lastNoNodeAvailableMs = now;
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();
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());
this.selector.send(metadataRequest.request());
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
log.debug("Initialize connection to node {} for sending metadata request", node.id());
initiateConnect(node, now);
@ -493,16 +499,17 @@ public class NetworkClient implements KafkaClient {
* Initiate a connection to the given node
*/
private void initiateConnect(Node node, long now) {
String nodeConnectionId = node.idString();
try {
log.debug("Initiating connection to node {} at {}:{}.", node.id(), node.host(), node.port());
this.connectionStates.connecting(node.id(), now);
selector.connect(node.id(),
this.connectionStates.connecting(nodeConnectionId, now);
selector.connect(nodeConnectionId,
new InetSocketAddress(node.host(), node.port()),
this.socketSendBuffer,
this.socketReceiveBuffer);
} catch (IOException e) {
/* attempt failed, we'll try again after the backoff */
connectionStates.disconnected(node.id());
connectionStates.disconnected(nodeConnectionId);
/* maybe the problem is our metadata, update it */
metadata.requestUpdate();
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;
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.consumer.internals.NoOpConsumerRebalanceCallback;
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.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
*/
@ -156,6 +156,9 @@ public class ConsumerConfig extends AbstractConfig {
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.";
/** <code>connections.max.idle.ms</code> */
public static final String CONNECTIONS_MAX_IDLE_MS_CONFIG = CommonClientConfigs.CONNECTIONS_MAX_IDLE_MS_CONFIG;
static {
CONFIG = new ConfigDef().define(BOOTSTRAP_SERVERS_CONFIG,
@ -277,7 +280,13 @@ public class ConsumerConfig extends AbstractConfig {
.define(VALUE_DESERIALIZER_CLASS_CONFIG,
Type.CLASS,
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,
@ -304,7 +313,7 @@ public class ConsumerConfig extends AbstractConfig {
return newProperties;
}
ConsumerConfig(Map<? extends Object, ? extends Object> props) {
ConsumerConfig(Map<?, ?> 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)
*/
public K key() throws Exception {
public K key() {
return key;
}
/**
* The value
*/
public V value() throws Exception {
public V value() {
return value;
}

View File

@ -12,15 +12,15 @@
*/
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.Collections;
import java.util.Iterator;
import java.util.List;
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
* 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.");
List<List<ConsumerRecord<K, V>>> recs = new ArrayList<List<ConsumerRecord<K, V>>>();
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());
}
return new ConcatenatedIterable<K, V>(recs);

View File

@ -24,6 +24,7 @@ import java.util.Properties;
import java.util.Set;
import java.util.concurrent.TimeUnit;
import java.util.concurrent.atomic.AtomicInteger;
import java.util.concurrent.atomic.AtomicReference;
import org.apache.kafka.clients.ClientUtils;
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.SubscriptionState;
import org.apache.kafka.common.Cluster;
import org.apache.kafka.common.KafkaException;
import org.apache.kafka.common.Metric;
import org.apache.kafka.common.PartitionInfo;
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 final Coordinator coordinator;
private final Deserializer<K> keyDeserializer;
private final Deserializer<V> valueDeserializer;
private final Fetcher<K, V> fetcher;
private final Time time;
@ -437,74 +441,98 @@ public class KafkaConsumer<K, V> implements Consumer<K, V> {
ConsumerRebalanceCallback callback,
Deserializer<K> keyDeserializer,
Deserializer<V> valueDeserializer) {
log.debug("Starting the Kafka consumer");
if (callback == null)
this.rebalanceCallback = config.getConfiguredInstance(ConsumerConfig.CONSUMER_REBALANCE_CALLBACK_CLASS_CONFIG,
ConsumerRebalanceCallback.class);
else
this.rebalanceCallback = callback;
this.time = new SystemTime();
this.autoCommit = config.getBoolean(ConsumerConfig.ENABLE_AUTO_COMMIT_CONFIG);
this.autoCommitIntervalMs = config.getLong(ConsumerConfig.AUTO_COMMIT_INTERVAL_MS_CONFIG);
this.lastCommitAttemptMs = time.milliseconds();
try {
log.debug("Starting the Kafka consumer");
if (callback == null)
this.rebalanceCallback = config.getConfiguredInstance(ConsumerConfig.CONSUMER_REBALANCE_CALLBACK_CLASS_CONFIG,
ConsumerRebalanceCallback.class);
else
this.rebalanceCallback = callback;
this.time = new SystemTime();
this.autoCommit = config.getBoolean(ConsumerConfig.ENABLE_AUTO_COMMIT_CONFIG);
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))
.timeWindow(config.getLong(ConsumerConfig.METRICS_SAMPLE_WINDOW_MS_CONFIG),
TimeUnit.MILLISECONDS);
String clientId = config.getString(ConsumerConfig.CLIENT_ID_CONFIG);
String jmxPrefix = "kafka.consumer";
if (clientId.length() <= 0)
clientId = "consumer-" + CONSUMER_CLIENT_ID_SEQUENCE.getAndIncrement();
List<MetricsReporter> reporters = config.getConfiguredInstances(ConsumerConfig.METRIC_REPORTER_CLASSES_CONFIG,
MetricsReporter.class);
reporters.add(new JmxReporter(jmxPrefix));
this.metrics = new Metrics(metricConfig, reporters, time);
this.retryBackoffMs = config.getLong(ConsumerConfig.RETRY_BACKOFF_MS_CONFIG);
this.metadata = new Metadata(retryBackoffMs, config.getLong(ConsumerConfig.METADATA_MAX_AGE_CONFIG));
List<InetSocketAddress> addresses = ClientUtils.parseAndValidateAddresses(config.getList(ConsumerConfig.BOOTSTRAP_SERVERS_CONFIG));
this.metadata.update(Cluster.bootstrap(addresses), 0);
MetricConfig metricConfig = new MetricConfig().samples(config.getInt(ConsumerConfig.METRICS_NUM_SAMPLES_CONFIG))
.timeWindow(config.getLong(ConsumerConfig.METRICS_SAMPLE_WINDOW_MS_CONFIG),
TimeUnit.MILLISECONDS);
String clientId = config.getString(ConsumerConfig.CLIENT_ID_CONFIG);
String jmxPrefix = "kafka.consumer";
if (clientId.length() <= 0)
clientId = "consumer-" + CONSUMER_CLIENT_ID_SEQUENCE.getAndIncrement();
List<MetricsReporter> reporters = config.getConfiguredInstances(ConsumerConfig.METRIC_REPORTER_CLASSES_CONFIG,
MetricsReporter.class);
reporters.add(new JmxReporter(jmxPrefix));
this.metrics = new Metrics(metricConfig, reporters, time);
this.retryBackoffMs = config.getLong(ConsumerConfig.RETRY_BACKOFF_MS_CONFIG);
this.metadata = new Metadata(retryBackoffMs, config.getLong(ConsumerConfig.METADATA_MAX_AGE_CONFIG));
List<InetSocketAddress> addresses = ClientUtils.parseAndValidateAddresses(config.getList(ConsumerConfig.BOOTSTRAP_SERVERS_CONFIG));
this.metadata.update(Cluster.bootstrap(addresses), 0);
String metricGrpPrefix = "consumer";
Map<String, String> metricsTags = new LinkedHashMap<String, String>();
metricsTags.put("client-id", clientId);
this.client = new NetworkClient(new Selector(metrics, time, metricGrpPrefix, metricsTags),
this.metadata,
clientId,
100, // a fixed large enough value will suffice
config.getLong(ConsumerConfig.RECONNECT_BACKOFF_MS_CONFIG),
config.getInt(ConsumerConfig.SEND_BUFFER_CONFIG),
config.getInt(ConsumerConfig.RECEIVE_BUFFER_CONFIG));
this.subscriptions = new SubscriptionState();
this.coordinator = new Coordinator(this.client,
config.getString(ConsumerConfig.GROUP_ID_CONFIG),
this.retryBackoffMs,
config.getLong(ConsumerConfig.SESSION_TIMEOUT_MS_CONFIG),
config.getString(ConsumerConfig.PARTITION_ASSIGNMENT_STRATEGY_CONFIG),
this.metadata,
this.subscriptions,
metrics,
metricGrpPrefix,
metricsTags,
this.time);
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(),
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);
String metricGrpPrefix = "consumer";
Map<String, String> metricsTags = new LinkedHashMap<String, String>();
metricsTags.put("client-id", clientId);
this.client = new NetworkClient(
new Selector(config.getLong(ConsumerConfig.CONNECTIONS_MAX_IDLE_MS_CONFIG), metrics, time, metricGrpPrefix, metricsTags),
this.metadata,
clientId,
100, // a fixed large enough value will suffice
config.getLong(ConsumerConfig.RECONNECT_BACKOFF_MS_CONFIG),
config.getInt(ConsumerConfig.SEND_BUFFER_CONFIG),
config.getInt(ConsumerConfig.RECEIVE_BUFFER_CONFIG));
this.subscriptions = new SubscriptionState();
this.coordinator = new Coordinator(this.client,
config.getString(ConsumerConfig.GROUP_ID_CONFIG),
this.retryBackoffMs,
config.getLong(ConsumerConfig.SESSION_TIMEOUT_MS_CONFIG),
config.getString(ConsumerConfig.PARTITION_ASSIGNMENT_STRATEGY_CONFIG),
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
public synchronized void close() {
log.trace("Closing the Kafka consumer.");
this.closed = true;
this.metrics.close();
this.client.close();
log.debug("The Kafka consumer has closed.");
close(false);
}
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) {
return this.autoCommit && this.lastCommitAttemptMs <= now - this.autoCommitIntervalMs;
}

View File

@ -96,7 +96,7 @@ public final class Coordinator {
this.time = time;
this.client = client;
this.generation = -1;
this.consumerId = "";
this.consumerId = JoinGroupRequest.UNKNOWN_CONSUMER_ID;
this.groupId = groupId;
this.metadata = metadata;
this.consumerCoordinator = null;
@ -120,29 +120,58 @@ public final class Coordinator {
// send a join group request to the coordinator
log.debug("(Re-)joining group {} with subscribed topics {}", groupId, subscribedTopics);
JoinGroupRequest request = new JoinGroupRequest(groupId,
(int) this.sessionTimeoutMs,
subscribedTopics,
this.consumerId,
this.assignmentStrategy);
ClientResponse resp = this.blockingCoordinatorRequest(ApiKeys.JOIN_GROUP, request.toStruct(), null, now);
// repeat processing the response until succeed or fatal error
do {
JoinGroupRequest request = new JoinGroupRequest(groupId,
(int) this.sessionTimeoutMs,
subscribedTopics,
this.consumerId,
this.assignmentStrategy);
// process the response
JoinGroupResponse response = new JoinGroupResponse(resp.responseBody());
// TODO: needs to handle disconnects and errors, should not just throw exceptions
Errors.forCode(response.errorCode()).maybeThrow();
this.consumerId = response.consumerId();
ClientResponse resp = this.blockingCoordinatorRequest(ApiKeys.JOIN_GROUP, request.toStruct(), null, now);
JoinGroupResponse response = new JoinGroupResponse(resp.responseBody());
short errorCode = response.errorCode();
// set the flag to refresh last committed offsets
this.subscriptions.needRefreshCommits();
if (errorCode == Errors.NONE.code()) {
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
this.sensors.partitionReassignments.record(time.milliseconds() - now);
log.debug("Joined group: {}", response);
// return assigned partitions
return response.assignedPartitions();
// record re-assignment time
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
boolean offsetsReady = true;
OffsetFetchResponse response = new OffsetFetchResponse(resp.responseBody());
// TODO: needs to handle disconnects
Map<TopicPartition, Long> offsets = new HashMap<TopicPartition, Long>(response.responseData().size());
for (Map.Entry<TopicPartition, OffsetFetchResponse.PartitionData> entry : response.responseData().entrySet()) {
TopicPartition tp = entry.getKey();
@ -238,7 +266,8 @@ public final class Coordinator {
// just ignore this partition
log.debug("Unknown topic or partition for " + tp);
} 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) {
// 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());
ConsumerMetadataRequest request = new ConsumerMetadataRequest(this.groupId);
RequestSend send = new RequestSend(node.id(),
RequestSend send = new RequestSend(node.idString(),
this.client.nextRequestHeader(ApiKeys.CONSUMER_METADATA),
request.toStruct());
long now = time.milliseconds();
@ -435,7 +464,7 @@ public final class Coordinator {
log.debug("Issuing request ({}: {}) to coordinator {}", api, request, this.consumerCoordinator.id());
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);
}
@ -470,9 +499,15 @@ public final class Coordinator {
if (response.errorCode() == Errors.NONE.code()) {
log.debug("Received successful heartbeat response.");
} 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();
} 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();
} else {
throw new KafkaException("Unexpected error in heartbeat response: "
@ -505,9 +540,10 @@ public final class Coordinator {
log.debug("Committed offset {} for partition {}", offset, tp);
subscriptions.committed(tp, offset);
} else if (errorCode == Errors.CONSUMER_COORDINATOR_NOT_AVAILABLE.code()
|| errorCode == Errors.NOT_COORDINATOR_FOR_CONSUMER.code()) {
|| errorCode == Errors.NOT_COORDINATOR_FOR_CONSUMER.code()) {
coordinatorDead();
} else {
// do not need to throw the exception but just log the error
log.error("Error committing partition {} at offset {}: {}",
tp,
offset,

View File

@ -124,7 +124,7 @@ public class Fetcher<K, V> {
*/
public void initFetches(Cluster cluster, long now) {
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)) {
log.trace("Initiating fetch to node {}: {}", node.id(), request);
client.send(request);
@ -209,12 +209,12 @@ public class Fetcher<K, V> {
} else if (this.client.ready(info.leader(), now)) {
Node node = info.leader();
ListOffsetRequest request = new ListOffsetRequest(-1, partitions);
RequestSend send = new RequestSend(node.id(),
RequestSend send = new RequestSend(node.idString(),
this.client.nextRequestHeader(ApiKeys.LIST_OFFSETS),
request.toStruct());
ClientRequest clientRequest = new ClientRequest(now, true, send, null);
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())
throw new IllegalStateException("This should not happen.");
ClientResponse response = responses.get(responses.size() - 1);
@ -231,13 +231,14 @@ public class Fetcher<K, V> {
log.debug("Fetched offset {} for partition {}", offset, topicPartition);
return offset;
} 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.",
topicPartition);
awaitMetadataUpdate();
} else {
// TODO: we should not just throw exceptions but should handle and log it.
Errors.forCode(errorCode).maybeThrow();
log.error("Attempt to fetch offsets for partition {} failed due to: {}",
topicPartition, Errors.forCode(errorCode).exception().getMessage());
awaitMetadataUpdate();
}
}
} else {
@ -257,7 +258,7 @@ public class Fetcher<K, V> {
for (TopicPartition partition : subscriptions.assignedPartitions()) {
Node node = cluster.leaderFor(partition);
// 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());
if (fetch == null) {
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()) {
int nodeId = entry.getKey();
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() {
public void onComplete(ClientResponse response) {
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
* 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.
@ -18,11 +18,11 @@ import java.util.concurrent.ExecutionException;
import java.util.concurrent.Future;
import java.util.concurrent.TimeUnit;
import java.util.concurrent.atomic.AtomicInteger;
import java.util.concurrent.atomic.AtomicReference;
import org.apache.kafka.clients.ClientUtils;
import org.apache.kafka.clients.Metadata;
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.Sender;
import org.apache.kafka.common.Cluster;
@ -72,11 +72,11 @@ import org.slf4j.LoggerFactory;
* props.put("buffer.memory", 33554432);
* props.put("key.serializer", "org.apache.kafka.common.serialization.StringSerializer");
* props.put("value.serializer", "org.apache.kafka.common.serialization.StringSerializer");
*
*
* Producer<String, String> producer = new KafkaProducer(props);
* for(int i = 0; i < 100; i++)
* producer.send(new ProducerRecord<String, String>("my-topic", Integer.toString(i), Integer.toString(i)));
*
*
* producer.close();
* }</pre>
* <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.
* <p>
* 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).
* <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
* generally have one of these buffers for each active partition).
* <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
* 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,
* 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
* records that arrive close together in time will generally batch together even with <code>linger.ms=0</code> so under heavy load
* 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,
* 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
* 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
* efficient requests when not under maximal load at the cost of a small amount of latency.
* <p>
* 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
* will cause the send call to result in an exception.
* <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
* 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
* string "42" or the integer 42).
* @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}.
* 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
* either the string "42" or the integer 42).
* @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
* 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
*/
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}.
* 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 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.
@ -191,81 +191,89 @@ public class KafkaProducer<K, V> implements Producer<K, V> {
@SuppressWarnings("unchecked")
private KafkaProducer(ProducerConfig config, Serializer<K> keySerializer, Serializer<V> valueSerializer) {
log.trace("Starting the Kafka producer");
this.producerConfig = config;
this.time = new SystemTime();
MetricConfig metricConfig = new MetricConfig().samples(config.getInt(ProducerConfig.METRICS_NUM_SAMPLES_CONFIG))
.timeWindow(config.getLong(ProducerConfig.METRICS_SAMPLE_WINDOW_MS_CONFIG),
TimeUnit.MILLISECONDS);
String clientId = config.getString(ProducerConfig.CLIENT_ID_CONFIG);
if (clientId.length() <= 0)
clientId = "producer-" + PRODUCER_CLIENT_ID_SEQUENCE.getAndIncrement();
String jmxPrefix = "kafka.producer";
List<MetricsReporter> reporters = config.getConfiguredInstances(ProducerConfig.METRIC_REPORTER_CLASSES_CONFIG,
MetricsReporter.class);
reporters.add(new JmxReporter(jmxPrefix));
this.metrics = new Metrics(metricConfig, reporters, time);
this.partitioner = new Partitioner();
long retryBackoffMs = config.getLong(ProducerConfig.RETRY_BACKOFF_MS_CONFIG);
this.metadataFetchTimeoutMs = config.getLong(ProducerConfig.METADATA_FETCH_TIMEOUT_CONFIG);
this.metadata = new Metadata(retryBackoffMs, config.getLong(ProducerConfig.METADATA_MAX_AGE_CONFIG));
this.maxRequestSize = config.getInt(ProducerConfig.MAX_REQUEST_SIZE_CONFIG);
this.totalMemorySize = config.getLong(ProducerConfig.BUFFER_MEMORY_CONFIG);
this.compressionType = CompressionType.forName(config.getString(ProducerConfig.COMPRESSION_TYPE_CONFIG));
Map<String, String> metricTags = new LinkedHashMap<String, String>();
metricTags.put("client-id", clientId);
this.accumulator = new RecordAccumulator(config.getInt(ProducerConfig.BATCH_SIZE_CONFIG),
this.totalMemorySize,
this.compressionType,
config.getLong(ProducerConfig.LINGER_MS_CONFIG),
retryBackoffMs,
config.getBoolean(ProducerConfig.BLOCK_ON_BUFFER_FULL_CONFIG),
metrics,
time,
metricTags);
List<InetSocketAddress> addresses = ClientUtils.parseAndValidateAddresses(config.getList(ProducerConfig.BOOTSTRAP_SERVERS_CONFIG));
this.metadata.update(Cluster.bootstrap(addresses), time.milliseconds());
try {
log.trace("Starting the Kafka producer");
this.producerConfig = config;
this.time = new SystemTime();
MetricConfig metricConfig = new MetricConfig().samples(config.getInt(ProducerConfig.METRICS_NUM_SAMPLES_CONFIG))
.timeWindow(config.getLong(ProducerConfig.METRICS_SAMPLE_WINDOW_MS_CONFIG),
TimeUnit.MILLISECONDS);
String clientId = config.getString(ProducerConfig.CLIENT_ID_CONFIG);
if (clientId.length() <= 0)
clientId = "producer-" + PRODUCER_CLIENT_ID_SEQUENCE.getAndIncrement();
String jmxPrefix = "kafka.producer";
List<MetricsReporter> reporters = config.getConfiguredInstances(ProducerConfig.METRIC_REPORTER_CLASSES_CONFIG,
MetricsReporter.class);
reporters.add(new JmxReporter(jmxPrefix));
this.metrics = new Metrics(metricConfig, reporters, time);
this.partitioner = config.getConfiguredInstance(ProducerConfig.PARTITIONER_CLASS_CONFIG, Partitioner.class);
long retryBackoffMs = config.getLong(ProducerConfig.RETRY_BACKOFF_MS_CONFIG);
this.metadataFetchTimeoutMs = config.getLong(ProducerConfig.METADATA_FETCH_TIMEOUT_CONFIG);
this.metadata = new Metadata(retryBackoffMs, config.getLong(ProducerConfig.METADATA_MAX_AGE_CONFIG));
this.maxRequestSize = config.getInt(ProducerConfig.MAX_REQUEST_SIZE_CONFIG);
this.totalMemorySize = config.getLong(ProducerConfig.BUFFER_MEMORY_CONFIG);
this.compressionType = CompressionType.forName(config.getString(ProducerConfig.COMPRESSION_TYPE_CONFIG));
Map<String, String> metricTags = new LinkedHashMap<String, String>();
metricTags.put("client-id", clientId);
this.accumulator = new RecordAccumulator(config.getInt(ProducerConfig.BATCH_SIZE_CONFIG),
this.totalMemorySize,
this.compressionType,
config.getLong(ProducerConfig.LINGER_MS_CONFIG),
retryBackoffMs,
config.getBoolean(ProducerConfig.BLOCK_ON_BUFFER_FULL_CONFIG),
metrics,
time,
metricTags);
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),
this.metadata,
clientId,
config.getInt(ProducerConfig.MAX_IN_FLIGHT_REQUESTS_PER_CONNECTION),
config.getLong(ProducerConfig.RECONNECT_BACKOFF_MS_CONFIG),
config.getInt(ProducerConfig.SEND_BUFFER_CONFIG),
config.getInt(ProducerConfig.RECEIVE_BUFFER_CONFIG));
this.sender = new Sender(client,
this.metadata,
this.accumulator,
config.getInt(ProducerConfig.MAX_REQUEST_SIZE_CONFIG),
(short) parseAcks(config.getString(ProducerConfig.ACKS_CONFIG)),
config.getInt(ProducerConfig.RETRIES_CONFIG),
config.getInt(ProducerConfig.TIMEOUT_CONFIG),
this.metrics,
new SystemTime(),
clientId);
String ioThreadName = "kafka-producer-network-thread" + (clientId.length() > 0 ? " | " + clientId : "");
this.ioThread = new KafkaThread(ioThreadName, this.sender, true);
this.ioThread.start();
NetworkClient client = new NetworkClient(
new Selector(config.getLong(ProducerConfig.CONNECTIONS_MAX_IDLE_MS_CONFIG), this.metrics, time, "producer", metricTags),
this.metadata,
clientId,
config.getInt(ProducerConfig.MAX_IN_FLIGHT_REQUESTS_PER_CONNECTION),
config.getLong(ProducerConfig.RECONNECT_BACKOFF_MS_CONFIG),
config.getInt(ProducerConfig.SEND_BUFFER_CONFIG),
config.getInt(ProducerConfig.RECEIVE_BUFFER_CONFIG));
this.sender = new Sender(client,
this.metadata,
this.accumulator,
config.getInt(ProducerConfig.MAX_REQUEST_SIZE_CONFIG),
(short) parseAcks(config.getString(ProducerConfig.ACKS_CONFIG)),
config.getInt(ProducerConfig.RETRIES_CONFIG),
config.getInt(ProducerConfig.TIMEOUT_CONFIG),
this.metrics,
new SystemTime(),
clientId);
String ioThreadName = "kafka-producer-network-thread" + (clientId.length() > 0 ? " | " + clientId : "");
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) {
this.keySerializer = config.getConfiguredInstance(ProducerConfig.KEY_SERIALIZER_CLASS_CONFIG,
Serializer.class);
this.keySerializer.configure(config.originals(), true);
} else {
this.keySerializer = keySerializer;
if (keySerializer == null) {
this.keySerializer = config.getConfiguredInstance(ProducerConfig.KEY_SERIALIZER_CLASS_CONFIG,
Serializer.class);
this.keySerializer.configure(config.originals(), true);
} else {
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) {
@ -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.
*/
@Override
@ -301,7 +309,7 @@ public class KafkaProducer<K, V> implements Producer<K, V> {
* or throw any exception that occurred while sending the record.
* <p>
* If you want to simulate a simple blocking call you can call the <code>get()</code> method immediately:
*
*
* <pre>
* {@code
* byte[] key = "key".getBytes();
@ -312,7 +320,7 @@ public class KafkaProducer<K, V> implements Producer<K, V> {
* <p>
* 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.
*
*
* <pre>
* {@code
* 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>
*
*
* 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>:
*
*
* <pre>
* {@code
* 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
* expensive callbacks it is recommended to use your own {@link java.util.concurrent.Executor} in the callback body
* to parallelize processing.
*
*
* @param record The record to send
* @param callback A user-supplied callback to execute when the record has been acknowledged by the server (null
* indicates no callback)
*
*
* @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 BufferExhaustedException If <code>block.on.buffer.full=false</code> and the buffer is full.
*
*
*/
@Override
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() +
" 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);
ensureValidRecordSize(serializedSize);
TopicPartition tp = new TopicPartition(record.topic(), partition);
@ -444,12 +452,12 @@ public class KafkaProducer<K, V> implements Producer<K, V> {
ProducerConfig.BUFFER_MEMORY_CONFIG +
" 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
* 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
* 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
* according to the <code>acks</code> configuration you have specified or else it results in an error.
* <p>
* 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();
* }
* </pre>
*
*
* 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.
*
*
* @throws InterruptException If the thread is interrupted while blocked
*/
@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.
* 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
*/
@Override
public void close() {
log.trace("Closing the Kafka producer.");
this.sender.initiateClose();
try {
this.ioThread.join();
} catch (InterruptedException e) {
throw new InterruptException(e);
close(Long.MAX_VALUE, TimeUnit.MILLISECONDS);
}
/**
* This method waits up to <code>timeout</code> for the producer to complete the sending of all incomplete requests.
* <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();
this.valueSerializer.close();
if (this.sender != null && this.ioThread != null && this.ioThread.isAlive()) {
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.");
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> {

View File

@ -24,11 +24,13 @@ import java.util.HashMap;
import java.util.List;
import java.util.Map;
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.Partitioner;
import org.apache.kafka.clients.producer.internals.DefaultPartitioner;
import org.apache.kafka.clients.producer.internals.ProduceRequestResult;
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
* 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 Partitioner partitioner = new Partitioner();
private final List<ProducerRecord<byte[], byte[]>> sent;
private final Partitioner partitioner;
private final List<ProducerRecord<K, V>> sent;
private final Deque<Completion> completions;
private boolean autoComplete;
private Map<TopicPartition, Long> offsets;
private final Serializer<K> keySerializer;
private final Serializer<V> valueSerializer;
/**
* Create a mock producer
*
*
* @param cluster The cluster holding metadata for this producer
* @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
* {@link #send(ProducerRecord) send()} to complete the call and unblock the @{link
* 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.autoComplete = autoComplete;
this.partitioner = partitioner;
this.keySerializer = keySerializer;
this.valueSerializer = valueSerializer;
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>();
}
/**
* Create a new mock producer with invented metadata the given autoComplete setting.
*
* Equivalent to {@link #MockProducer(Cluster, boolean) new MockProducer(null, autoComplete)}
* Create a new mock producer with invented metadata the given autoComplete setting and key\value serializers
*
* Equivalent to {@link #MockProducer(Cluster, boolean, Partitioner, Serializer, Serializer)} new MockProducer(Cluster.empty(), autoComplete, new DefaultPartitioner(), keySerializer, valueSerializer)}
*/
public MockProducer(boolean autoComplete) {
this(Cluster.empty(), autoComplete);
public MockProducer(boolean autoComplete, Serializer<K> keySerializer, Serializer<V> valueSerializer) {
this(Cluster.empty(), autoComplete, new DefaultPartitioner(), keySerializer, valueSerializer);
}
/**
* Create a new auto completing mock producer
*
* Equivalent to {@link #MockProducer(boolean) new MockProducer(true)}
* Create a new mock producer with invented metadata the given autoComplete setting, partitioner and key\value serializers
*
* Equivalent to {@link #MockProducer(Cluster, boolean, Partitioner, Serializer, Serializer)} new MockProducer(Cluster.empty(), autoComplete, partitioner, keySerializer, valueSerializer)}
*/
public MockProducer() {
this(true);
public MockProducer(boolean autoComplete, Partitioner partitioner, Serializer<K> keySerializer, Serializer<V> valueSerializer) {
this(Cluster.empty(), autoComplete, partitioner, keySerializer, valueSerializer);
}
/**
@ -87,20 +97,20 @@ public class MockProducer implements Producer<byte[], byte[]> {
* @see #history()
*/
@Override
public synchronized Future<RecordMetadata> send(ProducerRecord<byte[], byte[]> record) {
public synchronized Future<RecordMetadata> send(ProducerRecord<K, V> record) {
return send(record, null);
}
/**
* Adds the record to the list of sent records.
*
*
* @see #history()
*/
@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;
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();
FutureRecordMetadata future = new FutureRecordMetadata(result, 0);
TopicPartition topicPartition = new TopicPartition(record.topic(), partition);
@ -128,7 +138,7 @@ public class MockProducer implements Producer<byte[], byte[]> {
return offset;
}
}
public synchronized void flush() {
while (!this.completions.isEmpty())
completeNext();
@ -146,11 +156,15 @@ public class MockProducer implements Producer<byte[], byte[]> {
public void close() {
}
@Override
public void close(long timeout, TimeUnit timeUnit) {
}
/**
* Get the list of sent records since the last call to {@link #clear()}
*/
public synchronized List<ProducerRecord<byte[], byte[]>> history() {
return new ArrayList<ProducerRecord<byte[], byte[]>>(this.sent);
public synchronized List<ProducerRecord<K, V>> history() {
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.
*
*
* @return true if there was an uncompleted call to complete
*/
public synchronized boolean completeNext() {
@ -172,7 +186,7 @@ public class MockProducer implements Producer<byte[], byte[]> {
/**
* Complete the earliest uncompleted call with the given error.
*
*
* @return true if there was an uncompleted call to complete
*/
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 final long offset;
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.Map;
import java.util.concurrent.Future;
import java.util.concurrent.TimeUnit;
import org.apache.kafka.common.Metric;
import org.apache.kafka.common.PartitionInfo;
@ -67,4 +68,10 @@ public interface Producer<K, V> extends Closeable {
*/
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
* 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.
@ -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
* 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 {
@ -51,7 +51,7 @@ public class ProducerConfig extends AbstractConfig {
/** <code>metadata.max.age.ms</code> */
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;
/** <code>batch.size</code> */
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 "
@ -169,6 +169,13 @@ public class ProducerConfig extends AbstractConfig {
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.";
/** <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 {
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)
@ -217,7 +224,10 @@ public class ProducerConfig extends AbstractConfig {
Importance.LOW,
MAX_IN_FLIGHT_REQUESTS_PER_CONNECTION_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,
@ -242,7 +252,7 @@ public class ProducerConfig extends AbstractConfig {
return newProperties;
}
ProducerConfig(Map<? extends Object, ? extends Object> props) {
ProducerConfig(Map<?, ?> props) {
super(CONFIG, props);
}

View File

@ -17,9 +17,11 @@
package org.apache.kafka.clients.producer.internals;
import java.util.List;
import java.util.Map;
import java.util.Random;
import java.util.concurrent.atomic.AtomicInteger;
import org.apache.kafka.clients.producer.Partitioner;
import org.apache.kafka.common.Cluster;
import org.apache.kafka.common.PartitionInfo;
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 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());
/**
* 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 value is the original value bit AND against 0x7fffffff which is not its absolutely
* value.
*
*
* 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
* @return a positive 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.
*
*
* @param topic The topic name
* @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
*/
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);
int numPartitions = partitions.size();
if (partition != 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) {
if (keyBytes == null) {
int nextValue = counter.getAndIncrement();
List<PartitionInfo> availablePartitions = cluster.availablePartitionsForTopic(topic);
if (availablePartitions.size() > 0) {
int part = Partitioner.toPositive(nextValue) % availablePartitions.size();
int part = DefaultPartitioner.toPositive(nextValue) % availablePartitions.size();
return availablePartitions.get(part).partition();
} else {
// no partitions are available, give a non-available partition
return Partitioner.toPositive(nextValue) % numPartitions;
return DefaultPartitioner.toPositive(nextValue) % numPartitions;
}
} else {
// hash the key to choose a partition
return Partitioner.toPositive(Utils.murmur2(key)) % numPartitions;
// hash the keyBytes to choose a partition
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 volatile boolean closed;
private volatile AtomicInteger flushesInProgress;
private int drainIndex;
private final AtomicInteger flushesInProgress;
private final AtomicInteger appendsInProgress;
private final int batchSize;
private final CompressionType compression;
private final long lingerMs;
@ -67,6 +68,7 @@ public final class RecordAccumulator {
private final ConcurrentMap<TopicPartition, Deque<RecordBatch>> batches;
private final IncompleteRecordBatches incomplete;
/**
* Create a new record accumulator
*
@ -96,6 +98,7 @@ public final class RecordAccumulator {
this.drainIndex = 0;
this.closed = false;
this.flushesInProgress = new AtomicInteger(0);
this.appendsInProgress = new AtomicInteger(0);
this.batchSize = batchSize;
this.compression = compression;
this.lingerMs = lingerMs;
@ -146,40 +149,50 @@ public final class RecordAccumulator {
* @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 {
if (closed)
throw new IllegalStateException("Cannot send after the producer is closed.");
// check if we have an in-progress batch
Deque<RecordBatch> dq = dequeFor(tp);
synchronized (dq) {
RecordBatch last = dq.peekLast();
if (last != null) {
FutureRecordMetadata future = last.tryAppend(key, value, callback);
if (future != null)
return new RecordAppendResult(future, dq.size() > 1 || last.records.isFull(), false);
}
}
// we don't have an in-progress record batch try to allocate a new batch
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);
// We keep track of the number of appending thread to make sure we do not miss batches in
// abortIncompleteBatches().
appendsInProgress.incrementAndGet();
try {
if (closed)
throw new IllegalStateException("Cannot send after the producer is closed.");
// check if we have an in-progress batch
Deque<RecordBatch> dq = dequeFor(tp);
synchronized (dq) {
RecordBatch last = dq.peekLast();
if (last != null) {
FutureRecordMetadata future = last.tryAppend(key, value, callback);
if (future != null)
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);
// we don't have an in-progress record batch try to allocate a new batch
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) {
// 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) {
RecordBatch first = deque.peekFirst();
if (first != null) {
if (size + first.records.sizeInBytes() > maxSize && !ready.isEmpty()) {
// there is a rare case that a single batch size is larger than the request size due
// to compression; in this case we will still eventually send this batch in a single
// request
break;
} else {
RecordBatch batch = deque.pollFirst();
batch.records.close();
size += batch.records.sizeInBytes();
ready.add(batch);
batch.drainedMs = now;
boolean backoff = first.attempts > 0 && first.lastAttemptMs + retryBackoffMs > now;
// Only drain the batch if it is not during backoff period.
if (!backoff) {
if (size + first.records.sizeInBytes() > maxSize && !ready.isEmpty()) {
// there is a rare case that a single batch size is larger than the request size due
// to compression; in this case we will still eventually send this batch in a single
// request
break;
} else {
RecordBatch batch = deque.pollFirst();
batch.records.close();
size += batch.records.sizeInBytes();
ready.add(batch);
batch.drainedMs = now;
}
}
}
}
@ -347,7 +364,14 @@ public final class RecordAccumulator {
public void beginFlush() {
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
*/
@ -357,6 +381,40 @@ public final class RecordAccumulator {
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
*/
@ -399,7 +457,7 @@ public final class RecordAccumulator {
*/
private final static class IncompleteRecordBatches {
private final Set<RecordBatch> incomplete;
public IncompleteRecordBatches() {
this.incomplete = new HashSet<RecordBatch>();
}

View File

@ -83,6 +83,9 @@ public class Sender implements Runnable {
/* true while the sender thread is still running */
private volatile boolean running;
/* true when the caller wants to ignore all unsent/inflight messages and force close. */
private volatile boolean forceClose;
/* metrics */
private final SenderMetrics sensors;
@ -132,15 +135,23 @@ public class Sender implements Runnable {
// okay we stopped accepting requests but there may still be
// requests in the accumulator or waiting for acknowledgment,
// wait until these are completed.
while (this.accumulator.hasUnsent() || this.client.inFlightRequestCount() > 0) {
while (!forceClose && (this.accumulator.hasUnsent() || this.client.inFlightRequestCount() > 0)) {
try {
run(time.milliseconds());
} catch (Exception e) {
log.error("Uncaught error in kafka producer I/O thread: ", e);
}
}
this.client.close();
if (forceClose) {
// 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.");
}
@ -178,7 +189,6 @@ public class Sender implements Runnable {
now);
sensors.updateProduceRequestMetrics(batches);
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
// 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
@ -208,6 +218,14 @@ public class Sender implements Runnable {
this.wakeup();
}
/**
* Closes the sender without sending out any pending messages.
*/
public void forceClose() {
this.forceClose = true;
initiateClose();
}
/**
* Handle a produce response
*/
@ -302,7 +320,7 @@ public class Sender implements Runnable {
recordsByPartition.put(tp, batch);
}
ProduceRequest request = new ProduceRequest(acks, timeout, produceRecordsByPartition);
RequestSend send = new RequestSend(destination,
RequestSend send = new RequestSend(Integer.toString(destination),
this.client.nextRequestHeader(ApiKeys.PRODUCE),
request.toStruct());
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);
metrics.addMetric(m, new Measurable() {
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);
}
public void recordLatency(int node, long latency) {
public void recordLatency(String node, long latency) {
long now = time.milliseconds();
this.requestTimeSensor.record(latency, now);
if (node >= 0) {
if (!node.isEmpty()) {
String nodeTimeName = "node-" + node + ".latency";
Sensor nodeRequestTime = this.metrics.getSensor(nodeTimeName);
if (nodeRequestTime != null)

View File

@ -18,12 +18,14 @@ package org.apache.kafka.common;
public class Node {
private final int id;
private final String idString;
private final String host;
private final int port;
public Node(int id, String host, int port) {
super();
this.id = id;
this.idString = Integer.toString(id);
this.host = host;
this.port = port;
}
@ -39,6 +41,14 @@ public class Node {
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
*/

View File

@ -31,4 +31,9 @@ public class InterruptException extends KafkaException {
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
*/
public class UnknownTopicOrPartitionException extends RetriableException {
public class UnknownTopicOrPartitionException extends InvalidMetadataException {
private static final long serialVersionUID = 1L;

View File

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

View File

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

View File

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

View File

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

View File

@ -5,8 +5,8 @@
* 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
*
* 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,
@ -14,27 +14,17 @@
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.kafka.common.network;
package kafka.network
import org.apache.kafka.common.KafkaException;
import java.nio._
import java.nio.channels._
import kafka.utils._
public class InvalidReceiveException extends KafkaException {
@nonthreadsafe
private[kafka] class ByteBufferSend(val buffer: ByteBuffer) extends Send {
var complete: Boolean = false
public InvalidReceiveException(String message) {
super(message);
}
def this(size: Int) = this(ByteBuffer.allocate(size))
def writeTo(channel: GatheringByteChannel): Int = {
expectIncomplete()
var written = 0
written += channel.write(buffer)
if(!buffer.hasRemaining)
complete = true
written
}
public InvalidReceiveException(String message, Throwable cause) {
super(message, cause);
}
}

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.IOException;
import java.nio.ByteBuffer;
import java.nio.channels.ReadableByteChannel;
import java.nio.channels.ScatteringByteChannel;
/**
@ -22,24 +23,42 @@ import java.nio.channels.ScatteringByteChannel;
*/
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 int maxSize;
private ByteBuffer buffer;
public NetworkReceive(int source, ByteBuffer buffer) {
public NetworkReceive(String source, ByteBuffer buffer) {
this.source = source;
this.buffer = buffer;
this.size = null;
this.maxSize = UNLIMITED;
}
public NetworkReceive(int source) {
public NetworkReceive(String source) {
this.source = source;
this.size = ByteBuffer.allocate(4);
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
public int source() {
public String source() {
return source;
}
@ -48,13 +67,15 @@ public class NetworkReceive implements Receive {
return !size.hasRemaining() && !buffer.hasRemaining();
}
@Override
public ByteBuffer[] reify() {
return new ByteBuffer[] {this.buffer};
public long readFrom(ScatteringByteChannel channel) throws IOException {
return readFromReadableChannel(channel);
}
@Override
public long readFrom(ScatteringByteChannel channel) throws IOException {
// Need a method to read from ReadableByteChannel because BlockingChannel requires read with timeout
// See: http://stackoverflow.com/questions/2866557/timeout-for-socketchannel-doesnt-work
// This can go away after we get rid of BlockingChannel
@Deprecated
public long readFromReadableChannel(ReadableByteChannel channel) throws IOException {
int read = 0;
if (size.hasRemaining()) {
int bytesRead = channel.read(size);
@ -63,10 +84,12 @@ public class NetworkReceive implements Receive {
read += bytesRead;
if (!size.hasRemaining()) {
size.rewind();
int requestSize = size.getInt();
if (requestSize < 0)
throw new IllegalStateException("Invalid request (size = " + requestSize + ")");
this.buffer = ByteBuffer.allocate(requestSize);
int receiveSize = size.getInt();
if (receiveSize < 0)
throw new InvalidReceiveException("Invalid receive (size = " + receiveSize + ")");
if (maxSize != UNLIMITED && receiveSize > maxSize)
throw new InvalidReceiveException("Invalid receive (size = " + receiveSize + " larger than " + maxSize + ")");
this.buffer = ByteBuffer.allocate(receiveSize);
}
}
if (buffer != null) {
@ -83,4 +106,14 @@ public class NetworkReceive implements Receive {
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 NetworkSend(int destination, ByteBuffer... buffers) {
public NetworkSend(String destination, ByteBuffer... buffers) {
super(destination, sizeDelimit(buffers));
}

View File

@ -17,7 +17,6 @@
package org.apache.kafka.common.network;
import java.io.IOException;
import java.nio.ByteBuffer;
import java.nio.channels.ScatteringByteChannel;
/**
@ -28,18 +27,13 @@ public interface Receive {
/**
* The numeric id of the source from which we are receiving data.
*/
public int source();
public String source();
/**
* Are we done receiving data?
*/
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
* @param channel The channel to read from

View File

@ -29,12 +29,12 @@ public interface Selectable {
* @param receiveBufferSize The receive buffer for the socket
* @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
*/
public void disconnect(int id);
public void disconnect(String id);
/**
* 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
* @param send The request to send
*/
public void send(NetworkSend send);
public void send(Send send);
/**
* 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.
*/
public List<NetworkSend> completedSends();
public List<Send> completedSends();
/**
* 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()}
* 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()}
* call.
*/
public List<Integer> connected();
public List<String> connected();
/**
* Disable reads from the given 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
* @param id The id for the connection
*/
public void unmute(int id);
public void unmute(String id);
/**
* Disable reads from all connections

View File

@ -17,17 +17,8 @@ import java.io.IOException;
import java.net.ConnectException;
import java.net.InetSocketAddress;
import java.net.Socket;
import java.nio.channels.CancelledKeyException;
import java.nio.channels.SelectionKey;
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.nio.channels.*;
import java.util.*;
import java.util.concurrent.TimeUnit;
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.Max;
import org.apache.kafka.common.metrics.stats.Rate;
import org.apache.kafka.common.utils.SystemTime;
import org.apache.kafka.common.utils.Time;
import org.slf4j.Logger;
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>
* This class works with {@link NetworkSend} and {@link NetworkReceive} to transmit size-delimited network requests and
* responses.
* <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>
* 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>
*
* 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>
* 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>
*
* 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>.
*
* 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 final java.nio.channels.Selector selector;
private final Map<Integer, SelectionKey> keys;
private final List<NetworkSend> completedSends;
private final java.nio.channels.Selector nioSelector;
private final Map<String, SelectionKey> keys;
private final List<Send> completedSends;
private final List<NetworkReceive> completedReceives;
private final List<Integer> disconnected;
private final List<Integer> connected;
private final List<Integer> failedSends;
private final List<String> disconnected;
private final List<String> connected;
private final List<String> failedSends;
private final Time time;
private final SelectorMetrics sensors;
private final String metricGrpPrefix;
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 {
this.selector = java.nio.channels.Selector.open();
this.nioSelector = java.nio.channels.Selector.open();
} catch (IOException e) {
throw new KafkaException(e);
}
this.maxReceiveSize = maxReceiveSize;
this.connectionsMaxIdleNanos = connectionMaxIdleMs * 1000 * 1000;
this.time = time;
this.metricGrpPrefix = metricGrpPrefix;
this.metricTags = metricTags;
this.keys = new HashMap<Integer, SelectionKey>();
this.completedSends = new ArrayList<NetworkSend>();
this.keys = new HashMap<String, SelectionKey>();
this.completedSends = new ArrayList<Send>();
this.completedReceives = new ArrayList<NetworkReceive>();
this.connected = new ArrayList<Integer>();
this.disconnected = new ArrayList<Integer>();
this.failedSends = new ArrayList<Integer>();
this.connected = new ArrayList<String>();
this.disconnected = new ArrayList<String>();
this.failedSends = new ArrayList<String>();
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.
* <p>
* 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
*/
@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))
throw new IllegalStateException("There is already a connection for id " + id);
@ -143,7 +153,18 @@ public class Selector implements Selectable {
channel.close();
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));
this.keys.put(id, key);
}
@ -153,18 +174,18 @@ public class Selector implements Selectable {
* processed until the next {@link #poll(long, List) poll()} call.
*/
@Override
public void disconnect(int id) {
public void disconnect(String id) {
SelectionKey key = this.keys.get(id);
if (key != null)
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
public void wakeup() {
this.selector.wakeup();
this.nioSelector.wakeup();
}
/**
@ -172,12 +193,14 @@ public class Selector implements Selectable {
*/
@Override
public void close() {
for (SelectionKey key : this.selector.keys())
close(key);
List<String> connections = new LinkedList<String>(keys.keySet());
for (String id: connections)
close(id);
try {
this.selector.close();
this.nioSelector.close();
} 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
* @param send The request to send
*/
public void send(NetworkSend send) {
public void send(Send send) {
SelectionKey key = keyForId(send.destination());
Transmissions transmissions = transmissions(key);
if (transmissions.hasSend())
@ -194,7 +217,7 @@ public class Selector implements Selectable {
try {
key.interestOps(key.interestOps() | SelectionKey.OP_WRITE);
} catch (CancelledKeyException e) {
close(key);
close(transmissions.id);
this.failedSends.add(send.destination());
}
}
@ -220,10 +243,11 @@ public class Selector implements Selectable {
long startSelect = time.nanoseconds();
int readyKeys = select(timeout);
long endSelect = time.nanoseconds();
currentTimeNanos = endSelect;
this.sensors.selectTime.record(endSelect - startSelect, time.milliseconds());
if (readyKeys > 0) {
Set<SelectionKey> keys = this.selector.selectedKeys();
Set<SelectionKey> keys = this.nioSelector.selectedKeys();
Iterator<SelectionKey> iter = keys.iterator();
while (iter.hasNext()) {
SelectionKey key = iter.next();
@ -232,8 +256,9 @@ public class Selector implements Selectable {
Transmissions transmissions = transmissions(key);
SocketChannel channel = channel(key);
// register all per-broker metrics at once
sensors.maybeRegisterNodeMetrics(transmissions.id);
// register all per-connection metrics at once
sensors.maybeRegisterConnectionMetrics(transmissions.id);
lruConnections.put(transmissions.id, currentTimeNanos);
try {
/* 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 */
if (key.isReadable()) {
if (!transmissions.hasReceive())
transmissions.receive = new NetworkReceive(transmissions.id);
transmissions.receive.readFrom(channel);
transmissions.receive = new NetworkReceive(maxReceiveSize, transmissions.id);
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()) {
transmissions.receive.payload().rewind();
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 */
if (key.isWritable()) {
transmissions.send.writeTo(channel);
if (transmissions.send.remaining() <= 0) {
if (transmissions.send.completed()) {
this.completedSends.add(transmissions.send);
this.sensors.recordBytesSent(transmissions.id, transmissions.send.size());
transmissions.clearSend();
@ -270,7 +301,7 @@ public class Selector implements Selectable {
/* cancel any defunct sockets */
if (!key.isValid()) {
close(key);
close(transmissions.id);
this.disconnected.add(transmissions.id);
}
} catch (IOException e) {
@ -279,15 +310,16 @@ public class Selector implements Selectable {
log.info("Connection {} disconnected", desc);
else
log.warn("Error in I/O with connection to {}", desc, e);
close(key);
close(transmissions.id);
this.disconnected.add(transmissions.id);
}
}
}
long endIo = time.nanoseconds();
this.sensors.ioTime.record(endIo - endSelect, time.milliseconds());
maybeCloseOldestConnection();
}
private String socketDescription(SocketChannel channel) {
Socket socket = channel.socket();
if (socket == null)
@ -299,7 +331,7 @@ public class Selector implements Selectable {
}
@Override
public List<NetworkSend> completedSends() {
public List<Send> completedSends() {
return this.completedSends;
}
@ -309,17 +341,17 @@ public class Selector implements Selectable {
}
@Override
public List<Integer> disconnected() {
public List<String> disconnected() {
return this.disconnected;
}
@Override
public List<Integer> connected() {
public List<String> connected() {
return this.connected;
}
@Override
public void mute(int id) {
public void mute(String id) {
mute(this.keyForId(id));
}
@ -328,7 +360,7 @@ public class Selector implements Selectable {
}
@Override
public void unmute(int id) {
public void unmute(String id) {
unmute(this.keyForId(id));
}
@ -348,6 +380,27 @@ public class Selector implements Selectable {
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
*/
@ -369,17 +422,19 @@ public class Selector implements Selectable {
*/
private int select(long ms) throws IOException {
if (ms == 0L)
return this.selector.selectNow();
return this.nioSelector.selectNow();
else if (ms < 0L)
return this.selector.select();
return this.nioSelector.select();
else
return this.selector.select(ms);
return this.nioSelector.select(ms);
}
/**
* 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);
Transmissions trans = transmissions(key);
if (trans != null) {
@ -401,10 +456,10 @@ public class Selector implements Selectable {
/**
* 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);
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;
}
@ -426,11 +481,11 @@ public class Selector implements Selectable {
* The id and in-progress send and receive associated with a connection
*/
private static class Transmissions {
public int id;
public NetworkSend send;
public String id;
public Send send;
public NetworkReceive receive;
public Transmissions(int id) {
public Transmissions(String id) {
this.id = id;
}
@ -464,20 +519,27 @@ public class Selector implements Selectable {
public SelectorMetrics(Metrics metrics) {
this.metrics = 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);
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);
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);
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);
this.bytesSent.add(metricName, new Rate());
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);
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);
this.bytesReceived.add(metricName, new Rate());
metricName = new MetricName("response-rate", metricGrpName, "Responses received sent per second.", metricTags);
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);
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);
@ -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);
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);
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);
@ -515,17 +577,17 @@ public class Selector implements Selectable {
});
}
public void maybeRegisterNodeMetrics(int node) {
if (node >= 0) {
// if one sensor of the metrics has been registered for the node,
public void maybeRegisterConnectionMetrics(String connectionId) {
if (!connectionId.isEmpty() && metricsPerConnection) {
// if one sensor of the metrics has been registered for the connection,
// 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);
if (nodeRequest == null) {
String metricGrpName = metricGrpPrefix + "-node-metrics";
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);
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);
nodeRequest.add(metricName, new Max());
String nodeResponseName = "node-" + node + ".bytes-received";
String nodeResponseName = "node-" + connectionId + ".bytes-received";
Sensor nodeResponse = this.metrics.sensor(nodeResponseName);
metricName = new MetricName("incoming-byte-rate", metricGrpName, tags);
nodeResponse.add(metricName, new Rate());
metricName = new MetricName("response-rate", metricGrpName, "The average number of responses received per second.", tags);
nodeResponse.add(metricName, new Rate(new Count()));
String nodeTimeName = "node-" + node + ".latency";
String nodeTimeName = "node-" + connectionId + ".latency";
Sensor nodeRequestTime = this.metrics.sensor(nodeTimeName);
metricName = new MetricName("request-latency-avg", metricGrpName, tags);
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();
this.bytesSent.record(bytes, now);
if (node >= 0) {
String nodeRequestName = "node-" + node + ".bytes-sent";
if (!connectionId.isEmpty()) {
String nodeRequestName = "node-" + connectionId + ".bytes-sent";
Sensor nodeRequest = this.metrics.getSensor(nodeRequestName);
if (nodeRequest != null)
nodeRequest.record(bytes, now);
}
}
public void recordBytesReceived(int node, int bytes) {
public void recordBytesReceived(String connection, int bytes) {
long now = time.milliseconds();
this.bytesReceived.record(bytes, now);
if (node >= 0) {
String nodeRequestName = "node-" + node + ".bytes-received";
if (!connection.isEmpty()) {
String nodeRequestName = "node-" + connection + ".bytes-received";
Sensor nodeRequest = this.metrics.getSensor(nodeRequestName);
if (nodeRequest != null)
nodeRequest.record(bytes, now);

View File

@ -13,7 +13,6 @@
package org.apache.kafka.common.network;
import java.io.IOException;
import java.nio.ByteBuffer;
import java.nio.channels.GatheringByteChannel;
/**
@ -24,23 +23,13 @@ public interface Send {
/**
* The numeric id for the destination of this send
*/
public int destination();
/**
* The number of bytes remaining to send
*/
public int remaining();
public String destination();
/**
* Is this send complete?
*/
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
* to be completely written
@ -50,4 +39,9 @@ public interface Send {
*/
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,
new InvalidRequiredAcksException("Produce request specified an invalid value for required acks.")),
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<Short, Errors> codeToError = new HashMap<Short, Errors>();

View File

@ -118,6 +118,16 @@ public class Protocol {
new Field("offset",
INT64,
"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",
INT64,
"Timestamp of the commit"),
@ -125,7 +135,7 @@ public class Protocol {
STRING,
"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,
"Topic partition id."),
new Field("offset",
@ -149,6 +159,13 @@ public class Protocol {
new ArrayOf(OFFSET_COMMIT_REQUEST_PARTITION_V1),
"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",
STRING,
"The consumer group id."),
@ -166,7 +183,7 @@ public class Protocol {
STRING,
"The consumer id assigned by the group coordinator."),
new Field("topics",
new ArrayOf(OFFSET_COMMIT_REQUEST_TOPIC_V0),
new ArrayOf(OFFSET_COMMIT_REQUEST_TOPIC_V1),
"Topics to commit offsets."));
public static final Schema OFFSET_COMMIT_REQUEST_V2 = new Schema(new Field("group_id",
@ -182,7 +199,7 @@ public class Protocol {
INT64,
"Time period in ms to retain the offset."),
new Field("topics",
new ArrayOf(OFFSET_COMMIT_REQUEST_TOPIC_V1),
new ArrayOf(OFFSET_COMMIT_REQUEST_TOPIC_V2),
"Topics to commit offsets."));
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)));
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. */
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 */
/*
* 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",
INT32,
"Topic partition id."));
@ -239,8 +266,11 @@ public class Protocol {
public static final Schema OFFSET_FETCH_RESPONSE_V0 = new Schema(new Field("responses",
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_RESPONSE = new Schema[] {OFFSET_FETCH_RESPONSE_V0};
public static final Schema OFFSET_FETCH_REQUEST_V1 = OFFSET_FETCH_REQUEST_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 */
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) {
return CODE_TO_SECURITY_PROTOCOL.get(id).name;
return CODE_TO_SECURITY_PROTOCOL.get((short) id).name;
}
public static List<String> getNames() {

View File

@ -27,7 +27,10 @@ public class HeartbeatResponse extends AbstractRequestResponse {
/**
* 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;

View File

@ -30,7 +30,12 @@ public class JoinGroupResponse extends AbstractRequestResponse {
/**
* 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";

View File

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

View File

@ -25,7 +25,7 @@ public class RequestSend extends NetworkSend {
private final RequestHeader header;
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));
this.header = header;
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;
import java.io.Closeable;
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.
*/
public interface Deserializer<T> {
public interface Deserializer<T> extends Closeable {
/**
* Configure this class.
@ -38,8 +39,6 @@ public interface Deserializer<T> {
*/
public T deserialize(String topic, byte[] data);
/**
* Close this deserializer
*/
@Override
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;
import java.io.Closeable;
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.
*/
public interface Serializer<T> {
public interface Serializer<T> extends Closeable {
/**
* Configure this class.
@ -37,8 +38,12 @@ public interface Serializer<T> {
*/
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();
}

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) {
Thread thread = new Thread() {
public void run() {

View File

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

View File

@ -65,7 +65,7 @@ public class NetworkClientTest {
client.poll(1, time.milliseconds());
selector.clear();
assertTrue("Now the client is ready", client.ready(node, time.milliseconds()));
selector.disconnect(node.id());
selector.disconnect(node.idString());
client.poll(1, time.milliseconds());
selector.clear();
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)
public void testSendToUnreadyNode() {
RequestSend send = new RequestSend(5,
RequestSend send = new RequestSend("5",
client.nextRequestHeader(ApiKeys.METADATA),
new MetadataRequest(Arrays.asList("test")).toStruct());
ClientRequest request = new ClientRequest(time.milliseconds(), false, send, null);
@ -86,7 +86,7 @@ public class NetworkClientTest {
public void testSimpleRequestResponse() {
ProduceRequest produceRequest = new ProduceRequest((short) 1, 1000, Collections.<TopicPartition, ByteBuffer>emptyMap());
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();
ClientRequest request = new ClientRequest(time.milliseconds(), true, send, handler);
awaitReady(client, node);
@ -101,7 +101,7 @@ public class NetworkClientTest {
respHeader.writeTo(buffer);
resp.writeTo(buffer);
buffer.flip();
selector.completeReceive(new NetworkReceive(node.id(), buffer));
selector.completeReceive(new NetworkReceive(node.idString(), buffer));
List<ClientResponse> responses = client.poll(1, time.milliseconds());
assertEquals(1, responses.size());
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;
import static java.util.Arrays.asList;
import static java.util.Collections.singletonList;
import static org.junit.Assert.assertEquals;
import static org.junit.Assert.assertFalse;
import static org.junit.Assert.assertTrue;
import static org.junit.Assert.fail;
import java.util.ArrayList;
import java.util.concurrent.ExecutionException;
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;
public class MockProducerTest {
@ -34,23 +42,36 @@ public class MockProducerTest {
@Test
@SuppressWarnings("unchecked")
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());
Future<RecordMetadata> metadata = producer.send(record);
assertTrue("Send should be immediately complete", metadata.isDone());
assertFalse("Send should be successful", isError(metadata));
assertEquals("Offset should be 0", 0L, metadata.get().offset());
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();
assertEquals("Clear should erase our history", 0, producer.history().size());
}
@Test
public void testManualCompletion() throws Exception {
MockProducer producer = new MockProducer(false);
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());
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[]> record2 = new ProducerRecord<byte[], byte[]>(topic, "key2".getBytes(), "value2".getBytes());
Future<RecordMetadata> md1 = producer.send(record1);
assertFalse("Send shouldn't have completed", md1.isDone());
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
* 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.
@ -18,15 +18,15 @@ import static org.junit.Assert.assertTrue;
import java.util.List;
import org.apache.kafka.clients.producer.Partitioner;
import org.apache.kafka.common.Cluster;
import org.apache.kafka.common.Node;
import org.apache.kafka.common.PartitionInfo;
import org.junit.Test;
public class PartitionerTest {
private byte[] key = "key".getBytes();
private Partitioner partitioner = new Partitioner();
public class DefaultPartitionerTest {
private byte[] keyBytes = "key".getBytes();
private Partitioner partitioner = new DefaultPartitioner();
private Node node0 = new Node(0, "localhost", 99);
private Node node1 = new Node(1, "localhost", 100);
private Node node2 = new Node(2, "localhost", 101);
@ -38,15 +38,10 @@ public class PartitionerTest {
new PartitionInfo(topic, 0, node0, nodes, nodes));
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
public void testKeyPartitionIsStable() {
int partition = partitioner.partition("test", key, null, cluster);
assertEquals("Same key should yield same partition", 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", null, keyBytes, null, null, cluster));
}
@Test
@ -56,7 +51,7 @@ public class PartitionerTest {
int countForPart0 = 0;
int countForPart2 = 0;
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);
if (part == 0)
countForPart0++;

View File

@ -26,7 +26,10 @@ import java.util.LinkedHashMap;
import java.util.List;
import java.util.Map;
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.Node;
import org.apache.kafka.common.PartitionInfo;
@ -203,6 +206,44 @@ public class RecordAccumulatorTest {
// but have leaders with other sendable data.
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
public void testFlush() throws Exception {
@ -227,4 +268,27 @@ public class RecordAccumulatorTest {
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;
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.Type;
import org.apache.kafka.common.metrics.MetricsReporter;
import org.junit.Test;
import java.util.Map;
import java.util.Properties;
import static org.junit.Assert.fail;
public class AbstractConfigTest {
@Test
@ -73,7 +73,7 @@ public class AbstractConfigTest {
METRIC_REPORTER_CLASSES_DOC);
}
public TestConfig(Map<? extends Object, ? extends Object> props) {
public TestConfig(Map<?, ?> props) {
super(CONFIG, props);
}
}

View File

@ -22,13 +22,11 @@ import java.net.InetSocketAddress;
import java.net.ServerSocket;
import java.net.Socket;
import java.nio.ByteBuffer;
import java.util.ArrayList;
import java.util.Collections;
import java.util.LinkedHashMap;
import java.util.List;
import java.util.*;
import org.apache.kafka.common.metrics.Metrics;
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.test.TestUtils;
import org.junit.After;
@ -43,13 +41,15 @@ public class SelectorTest {
private static final int BUFFER_SIZE = 4 * 1024;
private EchoServer server;
private Time time;
private Selectable selector;
@Before
public void setup() throws Exception {
this.server = new EchoServer();
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
@ -63,7 +63,7 @@ public class SelectorTest {
*/
@Test
public void testServerDisconnect() throws Exception {
int node = 0;
String node = "0";
// connect and do a simple request
blockingConnect(node);
@ -84,7 +84,7 @@ public class SelectorTest {
*/
@Test
public void testClientDisconnect() throws Exception {
int node = 0;
String node = "0";
blockingConnect(node);
selector.disconnect(node);
selector.send(createSend(node, "hello1"));
@ -101,7 +101,7 @@ public class SelectorTest {
*/
@Test(expected = IllegalStateException.class)
public void testCantSendWithInProgress() throws Exception {
int node = 0;
String node = "0";
blockingConnect(node);
selector.send(createSend(node, "test1"));
selector.send(createSend(node, "test2"));
@ -113,7 +113,7 @@ public class SelectorTest {
*/
@Test(expected = IllegalStateException.class)
public void testCantSendWithoutConnecting() throws Exception {
selector.send(createSend(0, "test"));
selector.send(createSend("0", "test"));
selector.poll(1000L);
}
@ -122,7 +122,7 @@ public class SelectorTest {
*/
@Test(expected = IOException.class)
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
public void testConnectionRefused() throws Exception {
int node = 0;
String node = "0";
ServerSocket nonListeningSocket = new ServerSocket(0);
int nonListeningPort = nonListeningSocket.getLocalPort();
selector.connect(node, new InetSocketAddress("localhost", nonListeningPort), BUFFER_SIZE, BUFFER_SIZE);
@ -151,14 +151,15 @@ public class SelectorTest {
// create connections
InetSocketAddress addr = new InetSocketAddress("localhost", server.port);
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
int[] requests = new int[conns];
int[] responses = new int[conns];
Map<String, Integer> requests = new HashMap<String, Integer>();
Map<String, Integer> responses = new HashMap<String, Integer>();
int responseCount = 0;
for (int i = 0; i < conns; i++)
selector.send(createSend(i, i + "-" + 0));
for (int i = 0; i < conns; i++) {
String node = Integer.toString(i);
selector.send(createSend(node, node + "-0"));
}
// loop until we complete all requests
while (responseCount < conns * reqs) {
@ -171,19 +172,27 @@ public class SelectorTest {
for (NetworkReceive receive : selector.completedReceives()) {
String[] pieces = asString(receive).split("-");
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 the request counter", responses[receive.source()], Integer.parseInt(pieces[1]));
responses[receive.source()]++; // increment the expected counter
if (responses.containsKey(receive.source())) {
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++;
}
// prepare new sends for the next round
for (NetworkSend send : selector.completedSends()) {
int dest = send.destination();
requests[dest]++;
if (requests[dest] < reqs)
selector.send(createSend(dest, dest + "-" + requests[dest]));
for (Send send : selector.completedSends()) {
String dest = send.destination();
if (requests.containsKey(dest))
requests.put(dest, requests.get(dest) + 1);
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
public void testSendLargeRequest() throws Exception {
int node = 0;
String node = "0";
blockingConnect(node);
String big = TestUtils.randomString(10 * BUFFER_SIZE);
assertEquals(big, blockingRequest(node, big));
@ -204,41 +213,53 @@ public class SelectorTest {
*/
@Test
public void testEmptyRequest() throws Exception {
int node = 0;
String node = "0";
blockingConnect(node);
assertEquals("", blockingRequest(node, ""));
}
@Test(expected = IllegalStateException.class)
public void testExistingConnectionId() throws IOException {
blockingConnect(0);
blockingConnect(0);
blockingConnect("0");
blockingConnect("0");
}
@Test
public void testMute() throws Exception {
blockingConnect(0);
blockingConnect(1);
blockingConnect("0");
blockingConnect("1");
selector.send(createSend(0, "hello"));
selector.send(createSend(1, "hi"));
selector.send(createSend("0", "hello"));
selector.send(createSend("1", "hi"));
selector.mute(1);
selector.mute("1");
while (selector.completedReceives().isEmpty())
selector.poll(5);
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 {
selector.poll(5);
} while (selector.completedReceives().isEmpty());
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.poll(1000L);
while (true) {
@ -250,13 +271,13 @@ public class SelectorTest {
}
/* 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);
while (!selector.connected().contains(node))
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()));
}

View File

@ -43,8 +43,8 @@ public class ProtocolSerializationTest {
new Field("struct", new Schema(new Field("field", Type.INT32))));
this.struct = new Struct(this.schema).set("int8", (byte) 1)
.set("int16", (short) 1)
.set("int32", (int) 1)
.set("int64", (long) 1)
.set("int32", 1)
.set("int64", 1L)
.set("string", "1")
.set("bytes", "1".getBytes())
.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) {
Map<String, Object> serializerConfigs = new HashMap<String, Object>();
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.NetworkSend;
import org.apache.kafka.common.network.Selectable;
import org.apache.kafka.common.network.Send;
import org.apache.kafka.common.utils.Time;
/**
@ -28,23 +29,23 @@ import org.apache.kafka.common.utils.Time;
public class MockSelector implements Selectable {
private final Time time;
private final List<NetworkSend> initiatedSends = new ArrayList<NetworkSend>();
private final List<NetworkSend> completedSends = new ArrayList<NetworkSend>();
private final List<Send> initiatedSends = new ArrayList<Send>();
private final List<Send> completedSends = new ArrayList<Send>();
private final List<NetworkReceive> completedReceives = new ArrayList<NetworkReceive>();
private final List<Integer> disconnected = new ArrayList<Integer>();
private final List<Integer> connected = new ArrayList<Integer>();
private final List<String> disconnected = new ArrayList<String>();
private final List<String> connected = new ArrayList<String>();
public MockSelector(Time time) {
this.time = time;
}
@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);
}
@Override
public void disconnect(int id) {
public void disconnect(String id) {
this.disconnected.add(id);
}
@ -64,7 +65,7 @@ public class MockSelector implements Selectable {
}
@Override
public void send(NetworkSend send) {
public void send(Send send) {
this.initiatedSends.add(send);
}
@ -76,7 +77,7 @@ public class MockSelector implements Selectable {
}
@Override
public List<NetworkSend> completedSends() {
public List<Send> completedSends() {
return completedSends;
}
@ -94,21 +95,21 @@ public class MockSelector implements Selectable {
}
@Override
public List<Integer> disconnected() {
public List<String> disconnected() {
return disconnected;
}
@Override
public List<Integer> connected() {
public List<String> connected() {
return connected;
}
@Override
public void mute(int id) {
public void mute(String id) {
}
@Override
public void unmute(int id) {
public void unmute(String id) {
}
@Override

View File

@ -14,23 +14,32 @@
* See the License for the specific language governing permissions and
* 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;
/**
* A bucket of consumers that are scheduled for heartbeat expiration.
*
* 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) {
public class MockSerializer implements Serializer<byte[]> {
public static final AtomicInteger INIT_COUNT = new AtomicInteger(0);
public static final AtomicInteger CLOSE_COUNT = new AtomicInteger(0);
/* The list of consumers that are contained in this bucket */
val consumerRegistryList = new mutable.HashSet[ConsumerRegistry]
public MockSerializer() {
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);
if(_response != null) {
_respIterator = new ArrayList<ByteBufferMessageSet>(){{
add((ByteBufferMessageSet) _response.messageSet(_request.getTopic(), _request.getPartition()));
add(_response.messageSet(_request.getTopic(), _request.getPartition()));
}}.iterator();
}
_requestTime += (System.currentTimeMillis() - tempTime);

View File

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

View File

@ -317,12 +317,17 @@ object AdminUtils extends Logging {
if(str != null) {
Json.parseFull(str) match {
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)
map.get("config") match {
case Some(config: Map[String, String]) =>
for((k,v) <- config)
props.setProperty(k, v)
case Some(config: Map[_, _]) =>
for(configTup <- config)
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)
}

View File

@ -48,7 +48,7 @@ object ConsumerGroupCommand {
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 {
if (opts.options.has(opts.listOpt))
@ -174,7 +174,7 @@ object ConsumerGroupCommand {
val offsetMap = mutable.Map[TopicAndPartition, Long]()
val channel = ClientUtils.channelToOffsetManager(group, zkClient, channelSocketTimeoutMs, channelRetryBackoffMs)
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) =>
if (offsetAndMetadata == OffsetMetadataAndError.NoOffset) {

View File

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

View File

@ -38,7 +38,7 @@ object ReassignPartitionsCommand extends Logging {
CommandLineUtils.checkRequiredArgs(opts.parser, opts.options, 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 {
if(opts.options.has(opts.verifyOpt))
verifyAssignment(zkClient, opts)
@ -127,7 +127,7 @@ object ReassignPartitionsCommand extends Logging {
}
val reassignPartitionsCommand = new ReassignPartitionsCommand(zkClient, partitionsToBeReassigned.toMap)
// 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"
.format(ZkUtils.getPartitionReassignmentZkData(currentPartitionReplicaAssignment)))
// start the reassignment

View File

@ -47,7 +47,7 @@ object TopicCommand {
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 {
if(opts.options.has(opts.createOpt))
@ -143,7 +143,7 @@ object TopicCommand {
topics.foreach { topic =>
try {
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 {
ZkUtils.createPersistentPath(zkClient, ZkUtils.getDeleteTopicPath(topic))
println("Topic %s is marked for deletion.".format(topic))

View File

@ -18,9 +18,10 @@
package kafka.api
import java.nio.ByteBuffer
import kafka.network.{BoundedByteBufferSend, RequestChannel}
import kafka.network.RequestChannel.Response
import kafka.common.ErrorMapping
import kafka.network.{RequestOrResponseSend, RequestChannel}
import kafka.network.RequestChannel.Response
object ConsumerMetadataRequest {
val CurrentVersion = 0.shortValue
@ -64,7 +65,7 @@ case class ConsumerMetadataRequest(group: String,
override def handleError(e: Throwable, requestChannel: RequestChannel, request: RequestChannel.Request): Unit = {
// return ConsumerCoordinatorNotAvailable for all uncaught errors
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) = {

View File

@ -18,10 +18,9 @@
package kafka.api
import java.nio.ByteBuffer
import kafka.api.ApiUtils._
import collection.mutable.ListBuffer
import kafka.network.{BoundedByteBufferSend, RequestChannel}
import kafka.common.{TopicAndPartition, ErrorMapping}
import kafka.common.{ErrorMapping, TopicAndPartition}
import kafka.network.{RequestOrResponseSend, RequestChannel}
import kafka.network.RequestChannel.Response
import kafka.utils.Logging
@ -37,9 +36,9 @@ object ControlledShutdownRequest extends Logging {
}
}
case class ControlledShutdownRequest(val versionId: Short,
val correlationId: Int,
val brokerId: Int)
case class ControlledShutdownRequest(versionId: Short,
correlationId: Int,
brokerId: Int)
extends RequestOrResponse(Some(RequestKeys.ControlledShutdownKey)){
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 = {
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 = {
@ -74,4 +73,4 @@ case class ControlledShutdownRequest(val versionId: Short,
controlledShutdownRequest.append("; BrokerId: " + brokerId)
controlledShutdownRequest.toString()
}
}
}

View File

@ -39,9 +39,9 @@ object ControlledShutdownResponse {
}
case class ControlledShutdownResponse(val correlationId: Int,
val errorCode: Short = ErrorMapping.NoError,
val partitionsRemaining: Set[TopicAndPartition])
case class ControlledShutdownResponse(correlationId: Int,
errorCode: Short = ErrorMapping.NoError,
partitionsRemaining: Set[TopicAndPartition])
extends RequestOrResponse() {
def sizeInBytes(): Int ={
var size =
@ -68,4 +68,4 @@ case class ControlledShutdownResponse(val correlationId: Int,
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))
}
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 = {

View File

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

View File

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

View File

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

View File

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

View File

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

View File

@ -17,19 +17,16 @@
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 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 {
val CurrentVersion: Short = 0
val CurrentVersion: Short = 1
val DefaultClientId = ""
def readFrom(buffer: ByteBuffer): OffsetFetchRequest = {
@ -99,7 +96,7 @@ case class OffsetFetchRequest(groupId: String,
))
}.toMap
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 = {

View File

@ -18,9 +18,10 @@
package kafka.api
import java.nio.ByteBuffer
import kafka.common.{ErrorMapping, TopicAndPartition}
import kafka.api.ApiUtils._
import kafka.network.{BoundedByteBufferSend, RequestChannel}
import kafka.common.{ErrorMapping, TopicAndPartition}
import kafka.network.{RequestOrResponseSend, RequestChannel}
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))
}
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 = {

View File

@ -18,11 +18,12 @@
package kafka.api
import java.nio._
import kafka.message._
import kafka.api.ApiUtils._
import kafka.common._
import kafka.message._
import kafka.network.{RequestOrResponseSend, RequestChannel}
import kafka.network.RequestChannel.Response
import kafka.network.{RequestChannel, BoundedByteBufferSend}
object ProducerRequest {
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))
}
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 java.nio.ByteBuffer
import kafka.network.InvalidRequestException
object RequestKeys {
val ProduceKey: Short = 0
val FetchKey: Short = 1
@ -59,7 +61,7 @@ object RequestKeys {
def deserializerForKey(key: Short): (ByteBuffer) => RequestOrResponse = {
keyToNameAndDeserializerMap.get(key) match {
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 kafka.api.ApiUtils._
import kafka.network.{BoundedByteBufferSend, RequestChannel, InvalidRequestException}
import kafka.network.{RequestOrResponseSend, RequestChannel, InvalidRequestException}
import kafka.common.{TopicAndPartition, ErrorMapping}
import kafka.network.RequestChannel.Response
import kafka.utils.Logging
@ -106,7 +106,7 @@ case class StopReplicaRequest(versionId: Short,
case topicAndPartition => (topicAndPartition, ErrorMapping.codeFor(e.getClass.asInstanceOf[Class[Throwable]]))
}.toMap
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 = {

View File

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

View File

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

View File

@ -18,13 +18,15 @@
package kafka.api
import java.nio.ByteBuffer
import kafka.api.ApiUtils._
import collection.mutable.ListBuffer
import kafka.network.{BoundedByteBufferSend, RequestChannel}
import kafka.common.ErrorMapping
import kafka.network.{RequestOrResponseSend, RequestChannel}
import kafka.network.RequestChannel.Response
import kafka.utils.Logging
import scala.collection.mutable.ListBuffer
object TopicMetadataRequest extends Logging {
val CurrentVersion = 0.shortValue
val DefaultClientId = ""
@ -46,10 +48,10 @@ object TopicMetadataRequest extends Logging {
}
}
case class TopicMetadataRequest(val versionId: Short,
val correlationId: Int,
val clientId: String,
val topics: Seq[String])
case class TopicMetadataRequest(versionId: Short,
correlationId: Int,
clientId: String,
topics: Seq[String])
extends RequestOrResponse(Some(RequestKeys.MetadataKey)){
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]]))
}
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 = {
@ -93,4 +95,4 @@ case class TopicMetadataRequest(val versionId: Short,
topicMetadataRequest.append("; Topics: " + topics.mkString(","))
topicMetadataRequest.toString()
}
}
}

View File

@ -21,8 +21,8 @@ import java.nio.ByteBuffer
import kafka.api.ApiUtils._
import kafka.cluster.{Broker, BrokerEndPoint}
import kafka.common.{ErrorMapping, KafkaException, TopicAndPartition}
import kafka.network.{RequestOrResponseSend, RequestChannel}
import kafka.network.RequestChannel.Response
import kafka.network.{BoundedByteBufferSend, RequestChannel}
import org.apache.kafka.common.protocol.SecurityProtocol
import scala.collection.Set
@ -128,7 +128,7 @@ case class UpdateMetadataRequest (versionId: Short,
override def handleError(e: Throwable, requestChannel: RequestChannel, request: RequestChannel.Request): Unit = {
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 = {

View File

@ -74,7 +74,7 @@ object ClientUtils extends Logging{
} else {
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))
queryChannel.send(ConsumerMetadataRequest(group))
val response = queryChannel.receive()
val consumerMetadataResponse = ConsumerMetadataResponse.readFrom(response.buffer)
val consumerMetadataResponse = ConsumerMetadataResponse.readFrom(response.payload())
debug("Consumer metadata response: " + consumerMetadataResponse.toString)
if (consumerMetadataResponse.errorCode == ErrorMapping.NoError)
coordinatorOpt = consumerMetadataResponse.coordinatorOpt

View File

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

View File

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

View File

@ -23,4 +23,4 @@ package kafka.common
*/
class ConsumerRebalanceFailedException(message: String) extends RuntimeException(message) {
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,
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)
}

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. */
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 */
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 */
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 */
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() {
try {
debug("Clearing the current data chunk for this consumer iterator")
current.set(null)
}
debug("Clearing the current data chunk for this consumer iterator")
current.set(null)
}
}

View File

@ -17,10 +17,14 @@
package kafka.consumer
import java.nio.channels.ClosedByInterruptException
import kafka.api._
import kafka.network._
import kafka.utils._
import kafka.common.{ErrorMapping, TopicAndPartition}
import org.apache.kafka.common.network.{NetworkReceive, Receive}
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 {
var response: Receive = null
var response: NetworkReceive = null
try {
getOrMakeConnection()
blockingChannel.send(request)
response = blockingChannel.receive()
} catch {
case e : ClosedByInterruptException =>
throw e
case e : Throwable =>
info("Reconnect due to socket error: %s".format(e.toString))
// retry once
@ -89,12 +95,12 @@ class SimpleConsumer(val host: String,
def send(request: TopicMetadataRequest): TopicMetadataResponse = {
val response = sendRequest(request)
TopicMetadataResponse.readFrom(response.buffer)
TopicMetadataResponse.readFrom(response.payload())
}
def send(request: ConsumerMetadataRequest): ConsumerMetadataResponse = {
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
*/
def fetch(request: FetchRequest): FetchResponse = {
var response: Receive = null
var response: NetworkReceive = null
val specificTimer = fetchRequestAndResponseStats.getFetchRequestAndResponseStats(host, port).requestTimer
val aggregateTimer = fetchRequestAndResponseStats.getFetchRequestAndResponseAllBrokersStats.requestTimer
aggregateTimer.time {
@ -112,7 +118,7 @@ class SimpleConsumer(val host: String,
response = sendRequest(request)
}
}
val fetchResponse = FetchResponse.readFrom(response.buffer)
val fetchResponse = FetchResponse.readFrom(response.payload())
val fetchedSize = fetchResponse.sizeInBytes
fetchRequestAndResponseStats.getFetchRequestAndResponseStats(host, port).requestSizeHist.update(fetchedSize)
fetchRequestAndResponseStats.getFetchRequestAndResponseAllBrokersStats.requestSizeHist.update(fetchedSize)
@ -124,7 +130,7 @@ class SimpleConsumer(val host: String,
* @param request a [[kafka.api.OffsetRequest]] 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
@ -135,7 +141,7 @@ class SimpleConsumer(val host: String,
def commitOffsets(request: OffsetCommitRequest) = {
// TODO: With KAFKA-1012, we have to first issue a ConsumerMetadataRequest and connect to the coordinator before
// 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.
* @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() {
if(!isClosed && !blockingChannel.isConnected) {

View File

@ -178,7 +178,7 @@ private[kafka] class ZookeeperConsumerConnector(val config: ConsumerConfig,
private def connectZk() {
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.
@ -334,7 +334,7 @@ private[kafka] class ZookeeperConsumerConnector(val config: ConsumerConfig,
try {
kafkaCommitMeter.mark(offsetsToCommit.size)
offsetsChannel.send(offsetCommitRequest)
val offsetCommitResponse = OffsetCommitResponse.readFrom(offsetsChannel.receive().buffer)
val offsetCommitResponse = OffsetCommitResponse.readFrom(offsetsChannel.receive().payload())
trace("Offset commit response: %s.".format(offsetCommitResponse))
val (commitFailed, retryableIfFailed, shouldRefreshCoordinator, errorCount) = {
@ -421,7 +421,7 @@ private[kafka] class ZookeeperConsumerConnector(val config: ConsumerConfig,
ensureOffsetManagerConnected()
try {
offsetsChannel.send(offsetFetchRequest)
val offsetFetchResponse = OffsetFetchResponse.readFrom(offsetsChannel.receive().buffer)
val offsetFetchResponse = OffsetFetchResponse.readFrom(offsetsChannel.receive().payload())
trace("Offset fetch response: %s.".format(offsetFetchResponse))
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.
}
override def handleSessionEstablishmentError(error: Throwable): Unit = {
fatal("Could not establish session with zookeeper", error)
}
}
class ZKTopicPartitionChangeListener(val loadBalancerListener: ZKRebalancerListener)

View File

@ -18,7 +18,7 @@
package kafka.consumer
import scala.collection.JavaConversions._
import kafka.utils.{ZkUtils, ZKStringSerializer, Logging}
import kafka.utils.{ZkUtils, Logging}
import org.I0Itec.zkclient.{IZkStateListener, IZkChildListener, ZkClient}
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