diff --git a/clients/src/main/java/org/apache/kafka/clients/ClientRequest.java b/clients/src/main/java/org/apache/kafka/clients/ClientRequest.java deleted file mode 100644 index d32c319..0000000 --- a/clients/src/main/java/org/apache/kafka/clients/ClientRequest.java +++ /dev/null @@ -1,61 +0,0 @@ -/** - * Licensed to the Apache Software Foundation (ASF) under one or more contributor license agreements. See the NOTICE - * file distributed with this work for additional information regarding copyright ownership. The ASF licenses this file - * to You under the Apache License, Version 2.0 (the "License"); you may not use this file except in compliance with the - * License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software distributed under the License is distributed on - * an "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the License for the - * specific language governing permissions and limitations under the License. - */ -package org.apache.kafka.clients; - -import org.apache.kafka.common.requests.RequestSend; - -/** - * A request being sent to the server. This holds both the network send as well as the client-level metadata. - */ -public final class ClientRequest { - - private final long createdMs; - private final boolean expectResponse; - private final RequestSend request; - private final Object attachment; - - /** - * @param createdMs The unix timestamp in milliseconds for the time at which this request was created. - * @param expectResponse Should we expect a response message or is this request complete once it is sent? - * @param request The request - * @param attachment Associated data with the request - */ - public ClientRequest(long createdMs, boolean expectResponse, RequestSend request, Object attachment) { - this.createdMs = createdMs; - this.attachment = attachment; - this.request = request; - this.expectResponse = expectResponse; - } - - @Override - public String toString() { - return "ClientRequest(expectResponse=" + expectResponse + ", payload=" + attachment + ", request=" + request + ")"; - } - - public boolean expectResponse() { - return expectResponse; - } - - public RequestSend request() { - return request; - } - - public Object attachment() { - return attachment; - } - - public long createdTime() { - return createdMs; - } - -} \ No newline at end of file diff --git a/clients/src/main/java/org/apache/kafka/clients/ClientResponse.java b/clients/src/main/java/org/apache/kafka/clients/ClientResponse.java deleted file mode 100644 index 14ef69a..0000000 --- a/clients/src/main/java/org/apache/kafka/clients/ClientResponse.java +++ /dev/null @@ -1,78 +0,0 @@ -/** - * Licensed to the Apache Software Foundation (ASF) under one or more contributor license agreements. See the NOTICE - * file distributed with this work for additional information regarding copyright ownership. The ASF licenses this file - * to You under the Apache License, Version 2.0 (the "License"); you may not use this file except in compliance with the - * License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software distributed under the License is distributed on - * an "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the License for the - * specific language governing permissions and limitations under the License. - */ -package org.apache.kafka.clients; - -import org.apache.kafka.common.protocol.types.Struct; - -/** - * A response from the server. Contains both the body of the response as well as the correlated request that was - * originally sent. - */ -public class ClientResponse { - - private final long received; - private final boolean disconnected; - private final ClientRequest request; - private final Struct responseBody; - - /** - * @param request The original request - * @param received The unix timestamp when this response was received - * @param disconnected Whether the client disconnected before fully reading a response - * @param responseBody The response contents (or null) if we disconnected or no response was expected - */ - public ClientResponse(ClientRequest request, long received, boolean disconnected, Struct responseBody) { - super(); - this.received = received; - this.disconnected = disconnected; - this.request = request; - this.responseBody = responseBody; - } - - public long receivedTime() { - return received; - } - - public boolean wasDisconnected() { - return disconnected; - } - - public ClientRequest request() { - return request; - } - - public Struct responseBody() { - return responseBody; - } - - public boolean hasResponse() { - return responseBody != null; - } - - public long requestLatencyMs() { - return receivedTime() - this.request.createdTime(); - } - - @Override - public String toString() { - return "ClientResponse(received=" + received + - ", disconnected=" + - disconnected + - ", request=" + - request + - ", responseBody=" + - responseBody + - ")"; - } - -} diff --git a/clients/src/main/java/org/apache/kafka/clients/ClusterConnectionStates.java b/clients/src/main/java/org/apache/kafka/clients/ClusterConnectionStates.java deleted file mode 100644 index d304660..0000000 --- a/clients/src/main/java/org/apache/kafka/clients/ClusterConnectionStates.java +++ /dev/null @@ -1,113 +0,0 @@ -/** - * Licensed to the Apache Software Foundation (ASF) under one or more contributor license agreements. See the NOTICE - * file distributed with this work for additional information regarding copyright ownership. The ASF licenses this file - * to You under the Apache License, Version 2.0 (the "License"); you may not use this file except in compliance with the - * License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software distributed under the License is distributed on - * an "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the License for the - * specific language governing permissions and limitations under the License. - */ -package org.apache.kafka.clients; - -import java.util.HashMap; -import java.util.Map; - -/** - * The state of our connection to each node in the cluster. - * - */ -final class ClusterConnectionStates { - private final long reconnectBackoffMs; - private final Map nodeState; - - public ClusterConnectionStates(long reconnectBackoffMs) { - this.reconnectBackoffMs = reconnectBackoffMs; - this.nodeState = new HashMap(); - } - - /** - * Return true iff we can currently initiate a new connection to the given node. 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 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); - if (state == null) - return true; - else - return state.state == ConnectionState.DISCONNECTED && now - state.lastConnectAttemptMs >= this.reconnectBackoffMs; - } - - /** - * 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 now The current time in ms - */ - public boolean isBlackedOut(int node, long now) { - NodeConnectionState state = nodeState.get(node); - if (state == null) - return false; - else - return state.state == ConnectionState.DISCONNECTED && now - state.lastConnectAttemptMs < this.reconnectBackoffMs; - } - - /** - * Enter the connecting state for the given node. - * @param node The id of the node we are connecting to - * @param now The current time. - */ - public void connecting(int node, long now) { - nodeState.put(node, 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 - */ - public boolean isConnected(int node) { - NodeConnectionState state = nodeState.get(node); - 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 - */ - public boolean isConnecting(int node) { - NodeConnectionState state = nodeState.get(node); - return state != null && state.state == ConnectionState.CONNECTING; - } - - /** - * Enter the connected state for the given node - * @param node The node we have connected to - */ - public void connected(int node) { - nodeState(node).state = ConnectionState.CONNECTED; - } - - /** - * Enter the disconnected state for the given node - * @param node The node we have disconnected from - */ - public void disconnected(int node) { - nodeState(node).state = ConnectionState.DISCONNECTED; - } - - /** - * Get the state of our connection to the given state - * @param node The id of the node - * @return The state of our connection - */ - private NodeConnectionState nodeState(int node) { - NodeConnectionState state = this.nodeState.get(node); - if (state == null) - throw new IllegalStateException("No entry found for node " + node); - return state; - } -} \ No newline at end of file diff --git a/clients/src/main/java/org/apache/kafka/clients/ConnectionState.java b/clients/src/main/java/org/apache/kafka/clients/ConnectionState.java deleted file mode 100644 index ab7e322..0000000 --- a/clients/src/main/java/org/apache/kafka/clients/ConnectionState.java +++ /dev/null @@ -1,20 +0,0 @@ -/** - * Licensed to the Apache Software Foundation (ASF) under one or more contributor license agreements. See the NOTICE - * file distributed with this work for additional information regarding copyright ownership. The ASF licenses this file - * to You under the Apache License, Version 2.0 (the "License"); you may not use this file except in compliance with the - * License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software distributed under the License is distributed on - * an "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the License for the - * specific language governing permissions and limitations under the License. - */ -package org.apache.kafka.clients; - -/** - * The states of a node connection - */ -enum ConnectionState { - DISCONNECTED, CONNECTING, CONNECTED -} \ No newline at end of file diff --git a/clients/src/main/java/org/apache/kafka/clients/InFlightRequests.java b/clients/src/main/java/org/apache/kafka/clients/InFlightRequests.java deleted file mode 100644 index 936487b..0000000 --- a/clients/src/main/java/org/apache/kafka/clients/InFlightRequests.java +++ /dev/null @@ -1,126 +0,0 @@ -/** - * Licensed to the Apache Software Foundation (ASF) under one or more contributor license agreements. See the NOTICE - * file distributed with this work for additional information regarding copyright ownership. The ASF licenses this file - * to You under the Apache License, Version 2.0 (the "License"); you may not use this file except in compliance with the - * License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software distributed under the License is distributed on - * an "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the License for the - * specific language governing permissions and limitations under the License. - */ -package org.apache.kafka.clients; - -import java.util.ArrayDeque; -import java.util.Collections; -import java.util.Deque; -import java.util.HashMap; -import java.util.Map; - -/** - * The set of requests which have been sent or are being sent but haven't yet received a response - */ -final class InFlightRequests { - - private final int maxInFlightRequestsPerConnection; - private final Map> requests = new HashMap>(); - - public InFlightRequests(int maxInFlightRequestsPerConnection) { - this.maxInFlightRequestsPerConnection = maxInFlightRequestsPerConnection; - } - - /** - * Add the given request to the queue for the node it was directed to - */ - public void add(ClientRequest request) { - Deque reqs = this.requests.get(request.request().destination()); - if (reqs == null) { - reqs = new ArrayDeque(); - this.requests.put(request.request().destination(), reqs); - } - reqs.addFirst(request); - } - - /** - * Get the request queue for the given node - */ - private Deque requestQueue(int node) { - Deque reqs = requests.get(node); - if (reqs == null || reqs.isEmpty()) - throw new IllegalStateException("Response from server for which there are no in-flight requests."); - return reqs; - } - - /** - * Get the oldest request (the one that that will be completed next) for the given node - */ - public ClientRequest completeNext(int node) { - return requestQueue(node).pollLast(); - } - - /** - * 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) { - return requestQueue(node).peekFirst(); - } - - /** - * Complete the last request that was sent to a particular node. - * @param node The node the request was sent to - * @return The request - */ - public ClientRequest completeLastSent(int node) { - return requestQueue(node).pollFirst(); - } - - /** - * Can we send more requests to this node? - * - * @param node Node in question - * @return true iff we have no requests still being sent to the given node - */ - public boolean canSendMore(int node) { - Deque queue = requests.get(node); - return queue == null || queue.isEmpty() || - (queue.peekFirst().request().completed() && queue.size() < this.maxInFlightRequestsPerConnection); - } - - /** - * Return the number of inflight requests directed at the given node - * @param node The node - * @return The request count. - */ - public int inFlightRequestCount(int node) { - Deque queue = requests.get(node); - return queue == null ? 0 : queue.size(); - } - - /** - * Count all in-flight requests for all nodes - */ - public int inFlightRequestCount() { - int total = 0; - for (Deque deque : this.requests.values()) - total += deque.size(); - return total; - } - - /** - * Clear out all the in-flight requests for the given node and return them - * - * @param node The node - * @return All the in-flight requests for that node that have been removed - */ - public Iterable clearAll(int node) { - Deque reqs = requests.get(node); - if (reqs == null) { - return Collections.emptyList(); - } else { - return requests.remove(node); - } - } - -} \ No newline at end of file diff --git a/clients/src/main/java/org/apache/kafka/clients/KafkaClient.java b/clients/src/main/java/org/apache/kafka/clients/KafkaClient.java deleted file mode 100644 index 29658d4..0000000 --- a/clients/src/main/java/org/apache/kafka/clients/KafkaClient.java +++ /dev/null @@ -1,83 +0,0 @@ -/** - * Licensed to the Apache Software Foundation (ASF) under one or more contributor license agreements. See the NOTICE - * file distributed with this work for additional information regarding copyright ownership. The ASF licenses this file - * to You under the Apache License, Version 2.0 (the "License"); you may not use this file except in compliance with the - * License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software distributed under the License is distributed on - * an "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the License for the - * specific language governing permissions and limitations under the License. - */ -package org.apache.kafka.clients; - -import java.util.List; - -import org.apache.kafka.common.Node; -import org.apache.kafka.common.protocol.ApiKeys; -import org.apache.kafka.common.requests.RequestHeader; - -/** - * The interface for {@link NetworkClient} - */ -public interface KafkaClient { - - /** - * Check if we are currently ready to send another request to the given node but don't attempt to connect if we - * aren't. - * @param node The node to check - * @param now The current timestamp - */ - public boolean isReady(Node node, long now); - - /** - * Initiate a connection to the given node (if necessary), and return true if already connected. The readiness of a - * node will change only when poll is invoked. - * @param node The node to connect to. - * @param now The current time - * @return true iff we are ready to immediately initiate the sending of another request to the given node. - */ - public boolean ready(Node node, long now); - - /** - * Initiate the sending of the given requests and return any completed responses. Requests can only be sent on ready - * connections. - * @param requests The requests to send - * @param timeout The maximum amount of time to wait for responses in ms - * @param now The current time in ms - * @throws IllegalStateException If a request is sent to an unready node - */ - public List poll(List requests, long timeout, long now); - - /** - * Choose the node with the fewest outstanding requests. This method will prefer a node with an existing connection, - * but will potentially choose a node for which we don't yet have a connection if all existing connections are in - * use. - * @param now The current time in ms - * @return The node with the fewest in-flight requests. - */ - public Node leastLoadedNode(long now); - - /** - * The number of currently in-flight requests for which we have not yet returned a response - */ - public int inFlightRequestCount(); - - /** - * Generate a request header for the next request - * @param key The API key of the request - */ - public RequestHeader nextRequestHeader(ApiKeys key); - - /** - * Wake up the client if it is currently blocked waiting for I/O - */ - public void wakeup(); - - /** - * Close the client and disconnect from all nodes - */ - public void close(); - -} \ No newline at end of file diff --git a/clients/src/main/java/org/apache/kafka/clients/NetworkClient.java b/clients/src/main/java/org/apache/kafka/clients/NetworkClient.java deleted file mode 100644 index 522881c..0000000 --- a/clients/src/main/java/org/apache/kafka/clients/NetworkClient.java +++ /dev/null @@ -1,383 +0,0 @@ -/** - * Licensed to the Apache Software Foundation (ASF) under one or more contributor license agreements. See the NOTICE - * file distributed with this work for additional information regarding copyright ownership. The ASF licenses this file - * to You under the Apache License, Version 2.0 (the "License"); you may not use this file except in compliance with the - * License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software distributed under the License is distributed on - * an "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the License for the - * specific language governing permissions and limitations under the License. - */ -package org.apache.kafka.clients; - -import java.io.IOException; -import java.net.InetSocketAddress; -import java.util.ArrayList; -import java.util.List; -import java.util.Random; -import java.util.Set; - -import org.apache.kafka.clients.producer.internals.Metadata; -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.protocol.ApiKeys; -import org.apache.kafka.common.protocol.ProtoUtils; -import org.apache.kafka.common.protocol.types.Struct; -import org.apache.kafka.common.requests.MetadataRequest; -import org.apache.kafka.common.requests.MetadataResponse; -import org.apache.kafka.common.requests.RequestHeader; -import org.apache.kafka.common.requests.RequestSend; -import org.apache.kafka.common.requests.ResponseHeader; -import org.apache.kafka.common.utils.Utils; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; - -/** - * A network client for asynchronous request/response network i/o. This is an internal class used to implement the - * user-facing producer and consumer clients. - *

- * This class is not thread-safe! - */ -public class NetworkClient implements KafkaClient { - - private static final Logger log = LoggerFactory.getLogger(NetworkClient.class); - - /* the selector used to perform network i/o */ - private final Selectable selector; - - /* the current cluster metadata */ - private final Metadata metadata; - - /* the state of each node's connection */ - private final ClusterConnectionStates connectionStates; - - /* the set of requests currently being sent or awaiting a response */ - private final InFlightRequests inFlightRequests; - - /* the socket send buffer size in bytes */ - private final int socketSendBuffer; - - /* the socket receive size buffer in bytes */ - private final int socketReceiveBuffer; - - /* the client id used to identify this client in requests to the server */ - private final String clientId; - - /* a random offset to use when choosing nodes to avoid having all nodes choose the same node */ - private final int nodeIndexOffset; - - /* the current correlation id to use when sending requests to servers */ - private int correlation; - - /* true iff there is a metadata request that has been sent and for which we have not yet received a response */ - private boolean metadataFetchInProgress; - - public NetworkClient(Selectable selector, - Metadata metadata, - String clientId, - int maxInFlightRequestsPerConnection, - long reconnectBackoffMs, - int socketSendBuffer, - int socketReceiveBuffer) { - this.selector = selector; - this.metadata = metadata; - this.clientId = clientId; - this.inFlightRequests = new InFlightRequests(maxInFlightRequestsPerConnection); - this.connectionStates = new ClusterConnectionStates(reconnectBackoffMs); - this.socketSendBuffer = socketSendBuffer; - this.socketReceiveBuffer = socketReceiveBuffer; - this.correlation = 0; - this.nodeIndexOffset = new Random().nextInt(Integer.MAX_VALUE); - this.metadataFetchInProgress = false; - } - - /** - * Begin connecting to the given node, return true if we are already connected and ready to send to that node. - * @param node The node to check - * @param now The current timestamp - * @return True if we are ready to send to the given node - */ - @Override - public boolean ready(Node node, long now) { - if (isReady(node, now)) - return true; - - if (connectionStates.canConnect(node.id(), now)) - // if we are interested in sending to a node and we don't have a connection to it, initiate one - initiateConnect(node, now); - - return false; - } - - /** - * Check if the node with the given id is ready to send more requests. - * @param nodeId The node id - * @param now The current time in ms - * @return true if the node is ready - */ - @Override - public boolean isReady(Node node, long now) { - return isReady(node.id(), now); - } - - private boolean isReady(int node, long now) { - if (this.metadata.needsUpdate(now)) - // if we need to update our metadata declare all requests unready to metadata requests first priority - return false; - else - // otherwise we are ready if we are connected and can send more requests - return connectionStates.isConnected(node) && inFlightRequests.canSendMore(node); - } - - /** - * Initiate the given requests and check for any new responses, waiting up to the specified time. Requests can only - * be sent for ready nodes. - * @param requests The requests to initiate - * @param timeout The maximum amount of time to wait (in ms) for responses if there are none immediately - * @param now The current time in milliseconds - * @return The list of responses received - */ - @Override - public List poll(List requests, long timeout, long now) { - // should we update our metadata? - List sends = new ArrayList(); - maybeUpdateMetadata(sends, now); - - for (int i = 0; i < requests.size(); i++) { - ClientRequest request = requests.get(i); - int nodeId = request.request().destination(); - if (!isReady(nodeId, now)) - throw new IllegalStateException("Attempt to send a request to node " + nodeId + " which is not ready."); - - this.inFlightRequests.add(request); - sends.add(request.request()); - } - - // do the I/O - try { - this.selector.poll(timeout, sends); - } catch (IOException e) { - log.error("Unexpected error during I/O in producer network thread", e); - } - - List responses = new ArrayList(); - handleCompletedSends(responses, now); - handleCompletedReceives(responses, now); - handleDisconnections(responses, now); - handleConnections(); - - return responses; - } - - /** - * Get the number of in-flight requests - */ - @Override - public int inFlightRequestCount() { - return this.inFlightRequests.inFlightRequestCount(); - } - - /** - * Generate a request header for the given API key - * @param key The api key - * @return A request header with the appropriate client id and correlation id - */ - @Override - public RequestHeader nextRequestHeader(ApiKeys key) { - return new RequestHeader(key.id, clientId, correlation++); - } - - /** - * Interrupt the client if it is blocked waiting on I/O. - */ - @Override - public void wakeup() { - this.selector.wakeup(); - } - - /** - * Close the network client - */ - @Override - public void close() { - this.selector.close(); - } - - /** - * Choose the node with the fewest outstanding requests which is at least eligible for connection. This method will - * prefer a node with an existing connection, but will potentially choose a node for which we don't yet have a - * connection if all existing connections are in use. This method will never choose a node for which there is no - * existing connection and from which we have disconnected within the reconnect backoff period. - * @return The node with the fewest in-flight requests. - */ - public Node leastLoadedNode(long now) { - List nodes = this.metadata.fetch().nodes(); - int inflight = Integer.MAX_VALUE; - Node found = null; - 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())) { - // 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) { - // otherwise if this is the best we have found so far, record that - inflight = currInflight; - found = node; - } - } - - return found; - } - - /** - * Handle any completed request send. In particular if no response is expected consider the request complete. - * @param responses The list of responses to update - * @param now The current time - */ - private void handleCompletedSends(List responses, long now) { - // if no response is expected then when the send is completed, return it - for (NetworkSend send : this.selector.completedSends()) { - ClientRequest request = this.inFlightRequests.lastSent(send.destination()); - if (!request.expectResponse()) { - this.inFlightRequests.completeLastSent(send.destination()); - responses.add(new ClientResponse(request, now, false, null)); - } - } - } - - /** - * Handle any completed receives and update the response list with the responses received. - * @param responses The list of responses to update - * @param now The current time - */ - private void handleCompletedReceives(List responses, long now) { - for (NetworkReceive receive : this.selector.completedReceives()) { - int source = receive.source(); - ClientRequest req = inFlightRequests.completeNext(source); - ResponseHeader header = ResponseHeader.parse(receive.payload()); - short apiKey = req.request().header().apiKey(); - Struct body = (Struct) ProtoUtils.currentResponseSchema(apiKey).read(receive.payload()); - correlate(req.request().header(), header); - if (apiKey == ApiKeys.METADATA.id) { - handleMetadataResponse(req.request().header(), body, now); - } else { - // need to add body/header to response here - responses.add(new ClientResponse(req, now, false, body)); - } - } - } - - private void handleMetadataResponse(RequestHeader header, Struct body, long now) { - this.metadataFetchInProgress = false; - MetadataResponse response = new MetadataResponse(body); - Cluster cluster = response.cluster(); - // 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) - this.metadata.update(cluster, now); - else - log.trace("Ignoring empty metadata response with correlation id {}.", header.correlationId()); - } - - /** - * Handle any disconnected connections - * @param responses The list of responses that completed with the disconnection - * @param now The current time - */ - private void handleDisconnections(List responses, long now) { - for (int node : this.selector.disconnected()) { - connectionStates.disconnected(node); - log.debug("Node {} disconnected.", node); - for (ClientRequest request : this.inFlightRequests.clearAll(node)) { - log.trace("Cancelled request {} due to node {} being disconnected", request, node); - ApiKeys requestKey = ApiKeys.forId(request.request().header().apiKey()); - if (requestKey == ApiKeys.METADATA) - metadataFetchInProgress = false; - else - responses.add(new ClientResponse(request, now, true, null)); - } - } - // we got a disconnect so we should probably refresh our metadata and see if that broker is dead - if (this.selector.disconnected().size() > 0) - this.metadata.forceUpdate(); - } - - /** - * 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); - } - } - - /** - * Validate that the response corresponds to the request we expect or else explode - */ - private void correlate(RequestHeader requestHeader, ResponseHeader responseHeader) { - if (requestHeader.correlationId() != responseHeader.correlationId()) - throw new IllegalStateException("Correlation id for response (" + responseHeader.correlationId() + - ") does not match request (" + - requestHeader.correlationId() + - ")"); - } - - /** - * Create a metadata request for the given topics - */ - private ClientRequest metadataRequest(long now, int node, Set topics) { - MetadataRequest metadata = new MetadataRequest(new ArrayList(topics)); - RequestSend send = new RequestSend(node, nextRequestHeader(ApiKeys.METADATA), metadata.toStruct()); - return new ClientRequest(now, true, send, null); - } - - /** - * Add a metadata request to the list of sends if we need to make one - */ - private void maybeUpdateMetadata(List sends, long now) { - if (this.metadataFetchInProgress || !metadata.needsUpdate(now)) - return; - - Node node = this.leastLoadedNode(now); - if (node == null) - return; - - if (connectionStates.isConnected(node.id()) && inFlightRequests.canSendMore(node.id())) { - Set topics = metadata.topics(); - this.metadataFetchInProgress = true; - ClientRequest metadataRequest = metadataRequest(now, node.id(), topics); - log.debug("Sending metadata request {} to node {}", metadataRequest, node.id()); - sends.add(metadataRequest.request()); - this.inFlightRequests.add(metadataRequest); - } else if (connectionStates.canConnect(node.id(), now)) { - // we don't have a connection to this node right now, make one - initiateConnect(node, now); - } - } - - /** - * Initiate a connection to the given node - */ - private void initiateConnect(Node node, long now) { - try { - log.debug("Initiating connection to node {} at {}:{}.", node.id(), node.host(), node.port()); - selector.connect(node.id(), new InetSocketAddress(node.host(), node.port()), this.socketSendBuffer, this.socketReceiveBuffer); - this.connectionStates.connecting(node.id(), now); - } catch (IOException e) { - /* attempt failed, we'll try again after the backoff */ - connectionStates.disconnected(node.id()); - /* maybe the problem is our metadata, update it */ - metadata.forceUpdate(); - log.debug("Error connecting to node {} at {}:{}:", node.id(), node.host(), node.port(), e); - } - } - -} diff --git a/clients/src/main/java/org/apache/kafka/clients/NodeConnectionState.java b/clients/src/main/java/org/apache/kafka/clients/NodeConnectionState.java deleted file mode 100644 index 752a979..0000000 --- a/clients/src/main/java/org/apache/kafka/clients/NodeConnectionState.java +++ /dev/null @@ -1,31 +0,0 @@ -/** - * Licensed to the Apache Software Foundation (ASF) under one or more contributor license agreements. See the NOTICE - * file distributed with this work for additional information regarding copyright ownership. The ASF licenses this file - * to You under the Apache License, Version 2.0 (the "License"); you may not use this file except in compliance with the - * License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software distributed under the License is distributed on - * an "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the License for the - * specific language governing permissions and limitations under the License. - */ -package org.apache.kafka.clients; - -/** - * The state of our connection to a node - */ -final class NodeConnectionState { - - ConnectionState state; - long lastConnectAttemptMs; - - public NodeConnectionState(ConnectionState state, long lastConnectAttempt) { - this.state = state; - this.lastConnectAttemptMs = lastConnectAttempt; - } - - public String toString() { - return "NodeState(" + state + ", " + lastConnectAttemptMs + ")"; - } -} \ No newline at end of file diff --git a/clients/src/main/java/org/apache/kafka/clients/producer/KafkaProducer.java b/clients/src/main/java/org/apache/kafka/clients/producer/KafkaProducer.java index 00775ab..d15562a 100644 --- a/clients/src/main/java/org/apache/kafka/clients/producer/KafkaProducer.java +++ b/clients/src/main/java/org/apache/kafka/clients/producer/KafkaProducer.java @@ -21,7 +21,6 @@ import java.util.concurrent.ExecutionException; import java.util.concurrent.Future; import java.util.concurrent.TimeUnit; -import org.apache.kafka.clients.NetworkClient; import org.apache.kafka.clients.producer.internals.FutureRecordMetadata; import org.apache.kafka.clients.producer.internals.Metadata; import org.apache.kafka.clients.producer.internals.Partitioner; @@ -120,22 +119,19 @@ public class KafkaProducer implements Producer { metrics, time); List addresses = ClientUtils.parseAndValidateAddresses(config.getList(ProducerConfig.BOOTSTRAP_SERVERS_CONFIG)); - this.metadata.update(Cluster.bootstrap(addresses), 0); - - NetworkClient client = new NetworkClient(new Selector(this.metrics, time), - 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.update(Cluster.bootstrap(addresses), time.milliseconds()); + this.sender = new Sender(new Selector(this.metrics, time), this.metadata, this.accumulator, + clientId, config.getInt(ProducerConfig.MAX_REQUEST_SIZE_CONFIG), + config.getLong(ProducerConfig.RECONNECT_BACKOFF_MS_CONFIG), (short) parseAcks(config.getString(ProducerConfig.ACKS_CONFIG)), config.getInt(ProducerConfig.RETRIES_CONFIG), config.getInt(ProducerConfig.TIMEOUT_CONFIG), + config.getInt(ProducerConfig.SEND_BUFFER_CONFIG), + config.getInt(ProducerConfig.RECEIVE_BUFFER_CONFIG), + config.getInt(ProducerConfig.MAX_IN_FLIGHT_REQUESTS_PER_CONNECTION), this.metrics, new SystemTime()); this.ioThread = new KafkaThread("kafka-producer-network-thread", this.sender, true); diff --git a/clients/src/main/java/org/apache/kafka/clients/producer/internals/BufferPool.java b/clients/src/main/java/org/apache/kafka/clients/producer/internals/BufferPool.java index 169a656..d1d6c4b 100644 --- a/clients/src/main/java/org/apache/kafka/clients/producer/internals/BufferPool.java +++ b/clients/src/main/java/org/apache/kafka/clients/producer/internals/BufferPool.java @@ -16,19 +16,14 @@ */ package org.apache.kafka.clients.producer.internals; -import org.apache.kafka.clients.producer.BufferExhaustedException; -import org.apache.kafka.common.metrics.Metrics; -import org.apache.kafka.common.metrics.Sensor; -import org.apache.kafka.common.metrics.stats.Rate; -import org.apache.kafka.common.utils.Time; - import java.nio.ByteBuffer; import java.util.ArrayDeque; import java.util.Deque; -import java.util.concurrent.TimeUnit; import java.util.concurrent.locks.Condition; import java.util.concurrent.locks.ReentrantLock; +import org.apache.kafka.clients.producer.BufferExhaustedException; + /** * A pool of ByteBuffers kept under a given memory limit. This class is fairly specific to the needs of the producer. In @@ -49,9 +44,6 @@ public final class BufferPool { private final Deque free; private final Deque waiters; private long availableMemory; - private final Metrics metrics; - private final Time time; - private final Sensor waitTime; /** * Create a new buffer pool @@ -62,7 +54,7 @@ public final class BufferPool { * {@link #allocate(int)} call will block and wait for memory to be returned to the pool. If false * {@link #allocate(int)} will throw an exception if the buffer is out of memory. */ - public BufferPool(long memory, int poolableSize, boolean blockOnExhaustion, Metrics metrics, Time time) { + public BufferPool(long memory, int poolableSize, boolean blockOnExhaustion) { this.poolableSize = poolableSize; this.blockOnExhaustion = blockOnExhaustion; this.lock = new ReentrantLock(); @@ -70,13 +62,7 @@ public final class BufferPool { this.waiters = new ArrayDeque(); this.totalMemory = memory; this.availableMemory = memory; - this.metrics = metrics; - this.time = time; - this.waitTime = this.metrics.sensor("bufferpool-wait-time"); - this.waitTime.add("bufferpool-wait-ratio", - "The fraction of time an appender waits for space allocation.", - new Rate(TimeUnit.NANOSECONDS)); - } + } /** * Allocate a buffer of the given size. This method blocks if there is not enough memory and the buffer pool @@ -125,14 +111,7 @@ public final class BufferPool { // loop over and over until we have a buffer or have reserved // enough memory to allocate one while (accumulated < size) { - try { - long startWait = time.nanoseconds(); - moreMemory.await(300, TimeUnit.MILLISECONDS); - long endWait = time.nanoseconds(); - this.waitTime.record(endWait - startWait, time.milliseconds()); - } catch (InterruptedException e) { - // This should never happen. Just let it go. - } + moreMemory.await(); // check if we can satisfy this request from the free list, // otherwise allocate memory if (accumulated == 0 && size == this.poolableSize && !this.free.isEmpty()) { diff --git a/clients/src/main/java/org/apache/kafka/clients/producer/internals/Metadata.java b/clients/src/main/java/org/apache/kafka/clients/producer/internals/Metadata.java index 57bc285..f47a461 100644 --- a/clients/src/main/java/org/apache/kafka/clients/producer/internals/Metadata.java +++ b/clients/src/main/java/org/apache/kafka/clients/producer/internals/Metadata.java @@ -105,8 +105,8 @@ public final class Metadata { * since our last update and either (1) an update has been requested or (2) the current metadata has expired (more * than metadataExpireMs has passed since the last refresh) */ - public synchronized boolean needsUpdate(long now) { - long msSinceLastUpdate = now - this.lastRefreshMs; + public synchronized boolean needsUpdate(long nowMs) { + long msSinceLastUpdate = nowMs - this.lastRefreshMs; boolean updateAllowed = msSinceLastUpdate >= this.refreshBackoffMs; boolean updateNeeded = this.forceUpdate || msSinceLastUpdate >= this.metadataExpireMs; return updateAllowed && updateNeeded; @@ -129,9 +129,9 @@ public final class Metadata { /** * Update the cluster metadata */ - public synchronized void update(Cluster cluster, long now) { + public synchronized void update(Cluster cluster, long nowMs) { this.forceUpdate = false; - this.lastRefreshMs = now; + this.lastRefreshMs = nowMs; this.cluster = cluster; notifyAll(); log.debug("Updated cluster metadata to {}", cluster); diff --git a/clients/src/main/java/org/apache/kafka/clients/producer/internals/RecordAccumulator.java b/clients/src/main/java/org/apache/kafka/clients/producer/internals/RecordAccumulator.java index 1ed3c28..5ededcc 100644 --- a/clients/src/main/java/org/apache/kafka/clients/producer/internals/RecordAccumulator.java +++ b/clients/src/main/java/org/apache/kafka/clients/producer/internals/RecordAccumulator.java @@ -13,15 +13,7 @@ package org.apache.kafka.clients.producer.internals; import java.nio.ByteBuffer; -import java.util.ArrayDeque; -import java.util.ArrayList; -import java.util.Collections; -import java.util.Deque; -import java.util.HashMap; -import java.util.HashSet; -import java.util.List; -import java.util.Map; -import java.util.Set; +import java.util.*; import java.util.concurrent.ConcurrentMap; import org.apache.kafka.clients.producer.Callback; @@ -90,7 +82,7 @@ public final class RecordAccumulator { this.lingerMs = lingerMs; this.retryBackoffMs = retryBackoffMs; this.batches = new CopyOnWriteMap>(); - this.free = new BufferPool(totalSize, batchSize, blockOnBufferFull, metrics, time); + this.free = new BufferPool(totalSize, batchSize, blockOnBufferFull); this.time = time; registerMetrics(metrics); } @@ -99,21 +91,21 @@ public final class RecordAccumulator { metrics.addMetric("waiting-threads", "The number of user threads blocked waiting for buffer memory to enqueue their records", new Measurable() { - public double measure(MetricConfig config, long now) { + public double measure(MetricConfig config, long nowMs) { return free.queued(); } }); metrics.addMetric("buffer-total-bytes", "The maximum amount of buffer memory the client can use (whether or not it is currently used).", new Measurable() { - public double measure(MetricConfig config, long now) { + public double measure(MetricConfig config, long nowMs) { return free.totalMemory(); } }); metrics.addMetric("buffer-available-bytes", "The total amount of buffer memory that is not being used (either unallocated or in the free list).", new Measurable() { - public double measure(MetricConfig config, long now) { + public double measure(MetricConfig config, long nowMs) { return free.availableMemory(); } }); @@ -171,9 +163,9 @@ public final class RecordAccumulator { /** * Re-enqueue the given record batch in the accumulator to retry */ - public void reenqueue(RecordBatch batch, long now) { + public void reenqueue(RecordBatch batch, long nowMs) { batch.attempts++; - batch.lastAttemptMs = now; + batch.lastAttemptMs = nowMs; Deque deque = dequeFor(batch.topicPartition); synchronized (deque) { deque.addFirst(batch); @@ -183,8 +175,8 @@ public final class RecordAccumulator { /** * Get a list of nodes whose partitions are ready to be sent. *

- * A destination node is ready to send data if ANY one of its partition is not backing off the send and ANY of the - * following are true : + * A destination node is ready to send data if ANY one of its partition is not backing off the send + * and ANY of the following are true : *

    *
  1. The record set is full *
  2. The record set has sat in the accumulator for at least lingerMs milliseconds @@ -193,7 +185,7 @@ public final class RecordAccumulator { *
  3. The accumulator has been closed *
*/ - public Set ready(Cluster cluster, long now) { + public Set ready(Cluster cluster, long nowMs) { Set readyNodes = new HashSet(); boolean exhausted = this.free.queued() > 0; @@ -206,9 +198,9 @@ public final class RecordAccumulator { synchronized (deque) { RecordBatch batch = deque.peekFirst(); if (batch != null) { - boolean backingOff = batch.attempts > 0 && batch.lastAttemptMs + retryBackoffMs > now; + boolean backingOff = batch.attempts > 0 && batch.lastAttemptMs + retryBackoffMs > nowMs; boolean full = deque.size() > 1 || batch.records.isFull(); - boolean expired = now - batch.createdMs >= lingerMs; + boolean expired = nowMs - batch.createdMs >= lingerMs; boolean sendable = full || expired || exhausted || closed; if (sendable && !backingOff) readyNodes.add(leader); @@ -235,17 +227,18 @@ public final class RecordAccumulator { } /** - * Drain all the data for the given nodes and collate them into a list of batches that will fit within the specified - * size on a per-node basis. This method attempts to avoid choosing the same topic-node over and over. + * Drain all the data for the given nodes and collate them into a list of + * batches that will fit within the specified size on a per-node basis. + * This method attempts to avoid choosing the same topic-node over and over. * * @param cluster The current cluster metadata * @param nodes The list of node to drain * @param maxSize The maximum number of bytes to drain - * @param now The current unix time in milliseconds + * @param nowMs The current unix time in milliseconds * @return A list of {@link RecordBatch} for each node specified with total size less than the requested maxSize. * TODO: There may be a starvation issue due to iteration order */ - public Map> drain(Cluster cluster, Set nodes, int maxSize, long now) { + public Map> drain(Cluster cluster, Set nodes, int maxSize, long nowMs) { if (nodes.isEmpty()) return Collections.emptyMap(); @@ -273,7 +266,7 @@ public final class RecordAccumulator { batch.records.close(); size += batch.records.sizeInBytes(); ready.add(batch); - batch.drainedMs = now; + batch.drainedMs = nowMs; } } } diff --git a/clients/src/main/java/org/apache/kafka/clients/producer/internals/RecordBatch.java b/clients/src/main/java/org/apache/kafka/clients/producer/internals/RecordBatch.java index dd0af8a..5ee5455 100644 --- a/clients/src/main/java/org/apache/kafka/clients/producer/internals/RecordBatch.java +++ b/clients/src/main/java/org/apache/kafka/clients/producer/internals/RecordBatch.java @@ -42,9 +42,9 @@ public final class RecordBatch { private final ProduceRequestResult produceFuture; private final List thunks; - public RecordBatch(TopicPartition tp, MemoryRecords records, long now) { - this.createdMs = now; - this.lastAttemptMs = now; + public RecordBatch(TopicPartition tp, MemoryRecords records, long nowMs) { + this.createdMs = nowMs; + this.lastAttemptMs = nowMs; this.records = records; this.topicPartition = tp; this.produceFuture = new ProduceRequestResult(); diff --git a/clients/src/main/java/org/apache/kafka/clients/producer/internals/Sender.java b/clients/src/main/java/org/apache/kafka/clients/producer/internals/Sender.java index c67b947..9b1f565 100644 --- a/clients/src/main/java/org/apache/kafka/clients/producer/internals/Sender.java +++ b/clients/src/main/java/org/apache/kafka/clients/producer/internals/Sender.java @@ -12,15 +12,11 @@ */ package org.apache.kafka.clients.producer.internals; -import java.util.ArrayList; -import java.util.HashMap; -import java.util.List; -import java.util.Map; -import java.util.Set; - -import org.apache.kafka.clients.ClientRequest; -import org.apache.kafka.clients.ClientResponse; -import org.apache.kafka.clients.KafkaClient; +import java.io.IOException; +import java.net.InetSocketAddress; +import java.nio.ByteBuffer; +import java.util.*; + import org.apache.kafka.common.Cluster; import org.apache.kafka.common.Node; import org.apache.kafka.common.TopicPartition; @@ -33,11 +29,19 @@ import org.apache.kafka.common.metrics.Sensor; import org.apache.kafka.common.metrics.stats.Avg; import org.apache.kafka.common.metrics.stats.Max; import org.apache.kafka.common.metrics.stats.Rate; +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.protocol.ApiKeys; import org.apache.kafka.common.protocol.Errors; -import org.apache.kafka.common.requests.ProduceRequest; +import org.apache.kafka.common.protocol.ProtoUtils; +import org.apache.kafka.common.protocol.types.Struct; +import org.apache.kafka.common.requests.MetadataRequest; +import org.apache.kafka.common.requests.MetadataResponse; import org.apache.kafka.common.requests.ProduceResponse; +import org.apache.kafka.common.requests.RequestHeader; import org.apache.kafka.common.requests.RequestSend; +import org.apache.kafka.common.requests.ResponseHeader; import org.apache.kafka.common.utils.Time; import org.apache.kafka.common.utils.Utils; import org.slf4j.Logger; @@ -52,13 +56,16 @@ public class Sender implements Runnable { private static final Logger log = LoggerFactory.getLogger(Sender.class); /* the state of each nodes connection */ - private final KafkaClient client; + private final NodeStates nodeStates; /* the record accumulator that batches records */ private final RecordAccumulator accumulator; - /* the metadata for the client */ - private final Metadata metadata; + /* the selector used to perform network i/o */ + private final Selectable selector; + + /* the client id used to identify this client in requests to the server */ + private final String clientId; /* the maximum request size to attempt to send to the server */ private final int maxRequestSize; @@ -72,33 +79,67 @@ public class Sender implements Runnable { /* the number of times to retry a failed request before giving up */ private final int retries; + /* the socket send buffer size in bytes */ + private final int socketSendBuffer; + + /* the socket receive size buffer in bytes */ + private final int socketReceiveBuffer; + + /* the set of currently in-flight requests awaiting a response from the server */ + private final InFlightRequests inFlightRequests; + + /* a reference to the current Cluster instance */ + private final Metadata metadata; + /* the clock instance used for getting the time */ private final Time time; + /* the current node to attempt to use for metadata requests (will round-robin over nodes) */ + private int metadataFetchNodeIndex; + + /* the current correlation id to use when sending requests to servers */ + private int correlation; + + /* true iff there is a metadata request that has been sent and for which we have not yet received a response */ + private boolean metadataFetchInProgress; + /* true while the sender thread is still running */ private volatile boolean running; /* metrics */ private final SenderMetrics sensors; - public Sender(KafkaClient client, + public Sender(Selectable selector, Metadata metadata, RecordAccumulator accumulator, + String clientId, int maxRequestSize, + long reconnectBackoffMs, short acks, int retries, int requestTimeout, + int socketSendBuffer, + int socketReceiveBuffer, + int maxInFlightRequestsPerConnection, Metrics metrics, Time time) { - this.client = client; + this.nodeStates = new NodeStates(reconnectBackoffMs); this.accumulator = accumulator; - this.metadata = metadata; + this.selector = selector; this.maxRequestSize = maxRequestSize; + this.metadata = metadata; + this.clientId = clientId; this.running = true; this.requestTimeout = requestTimeout; this.acks = acks; this.retries = retries; + this.socketSendBuffer = socketSendBuffer; + this.socketReceiveBuffer = socketReceiveBuffer; + this.inFlightRequests = new InFlightRequests(maxInFlightRequestsPerConnection); + this.correlation = 0; + this.metadataFetchInProgress = false; this.time = time; + this.metadataFetchNodeIndex = new Random().nextInt(); this.sensors = new SenderMetrics(metrics); } @@ -128,9 +169,10 @@ public class Sender implements Runnable { } catch (Exception e) { log.error("Uncaught error in kafka producer I/O thread: ", e); } - } while (this.accumulator.hasUnsent() || this.client.inFlightRequestCount() > 0); + } while (this.accumulator.hasUnsent() || this.inFlightRequests.totalInFlightRequests() > 0); - this.client.close(); + // close all the connections + this.selector.close(); log.debug("Shutdown of Kafka producer I/O thread has completed."); } @@ -138,36 +180,117 @@ public class Sender implements Runnable { /** * Run a single iteration of sending * - * @param now The current POSIX time in milliseconds + * @param nowMs The current POSIX time in milliseconds */ - public void run(long now) { + public void run(long nowMs) { Cluster cluster = metadata.fetch(); // get the list of partitions with data ready to send - Set ready = this.accumulator.ready(cluster, now); + Set ready = this.accumulator.ready(cluster, nowMs); - // remove any nodes we aren't ready to send to - for (Node node : ready) { - if (!this.client.ready(node, now)) - ready.remove(node); - } + // should we update our metadata? + List sends = new ArrayList(); + maybeUpdateMetadata(cluster, sends, nowMs); + + // prune the list of ready nodes to eliminate any that we aren't ready to send yet + Set sendable = processReadyNode(ready, nowMs); // create produce requests - Map> batches = this.accumulator.drain(cluster, ready, this.maxRequestSize, now); - List requests = createProduceRequests(batches, now); + Map> batches = this.accumulator.drain(cluster, sendable, this.maxRequestSize, nowMs); + List requests = generateProduceRequests(batches, nowMs); sensors.updateProduceRequestMetrics(requests); if (ready.size() > 0) { - log.trace("Nodes with data ready to send: {}", ready); + log.trace("Partitions with complete batches: {}", ready); + log.trace("Partitions ready to initiate a request: {}", sendable); log.trace("Created {} produce requests: {}", requests.size(), requests); } - List responses = this.client.poll(requests, 100L, now); - for (ClientResponse response : responses) { - if (response.wasDisconnected()) - handleDisconnect(response, now); - else - handleResponse(response, now); + for (int i = 0; i < requests.size(); i++) { + InFlightRequest request = requests.get(i); + this.inFlightRequests.add(request); + sends.add(request.request); + } + + // do the I/O + try { + this.selector.poll(100L, sends); + } catch (IOException e) { + log.error("Unexpected error during I/O in producer network thread", e); + } + + // handle responses, connections, and disconnections + handleSends(this.selector.completedSends()); + handleResponses(this.selector.completedReceives(), nowMs); + handleDisconnects(this.selector.disconnected(), nowMs); + handleConnects(this.selector.connected()); + } + + /** + * Add a metadata request to the list of sends if we need to make one + */ + private void maybeUpdateMetadata(Cluster cluster, List sends, long nowMs) { + if (this.metadataFetchInProgress || !metadata.needsUpdate(nowMs)) + return; + + Node node = selectMetadataDestination(cluster); + if (node == null) + return; + + if (nodeStates.isConnected(node.id())) { + Set topics = metadata.topics(); + this.metadataFetchInProgress = true; + InFlightRequest metadataRequest = metadataRequest(nowMs, node.id(), topics); + log.debug("Sending metadata request {} to node {}", metadataRequest, node.id()); + sends.add(metadataRequest.request); + this.inFlightRequests.add(metadataRequest); + } else if (nodeStates.canConnect(node.id(), nowMs)) { + // we don't have a connection to this node right now, make one + initiateConnect(node, nowMs); + } + } + + /** + * Find a good node to make a metadata request to. This method will first look for a node that has an existing + * connection and no outstanding requests. If there are no such nodes it will look for a node with no outstanding + * requests. + * @return A node with no requests currently being sent or null if no such node exists + */ + private Node selectMetadataDestination(Cluster cluster) { + List nodes = cluster.nodes(); + + // first look for a node to which we are connected and have no outstanding requests + boolean connectionInProgress = false; + for (int i = 0; i < nodes.size(); i++) { + Node node = nodes.get(metadataNodeIndex(i, nodes.size())); + if (nodeStates.isConnected(node.id()) && this.inFlightRequests.canSendMore(node.id())) { + this.metadataFetchNodeIndex = metadataNodeIndex(i + 1, nodes.size()); + return node; + } else if (nodeStates.isConnecting(node.id())) { + connectionInProgress = true; + } + } + + // if we have a connection that is being established now, just wait for that don't make another + if (connectionInProgress) + return null; + + // okay, no luck, pick a random unused node + for (int i = 0; i < nodes.size(); i++) { + Node node = nodes.get(metadataNodeIndex(i, nodes.size())); + if (this.inFlightRequests.canSendMore(node.id())) { + this.metadataFetchNodeIndex = metadataNodeIndex(i + 1, nodes.size()); + return node; + } } + + return null; // we failed to find a good destination + } + + /** + * Get the index in the node list of the node to use for the metadata request + */ + private int metadataNodeIndex(int offset, int size) { + return Utils.abs(offset + this.metadataFetchNodeIndex) % size; } /** @@ -179,40 +302,161 @@ public class Sender implements Runnable { this.wakeup(); } - private void handleDisconnect(ClientResponse response, long now) { - log.trace("Cancelled request {} due to node {} being disconnected", response, response.request().request().destination()); - int correlation = response.request().request().header().correlationId(); - @SuppressWarnings("unchecked") - Map responseBatches = (Map) response.request().attachment(); - for (RecordBatch batch : responseBatches.values()) - completeBatch(batch, Errors.NETWORK_EXCEPTION, -1L, correlation, now); + /** + * Process the set of destination nodes with data ready to send. + * + * 1) If we have an unknown leader node, force refresh the metadata. + * 2) If we have a connection to the appropriate node, add + * it to the returned set; + * 3) If we have not a connection yet, initialize one + */ + private Set processReadyNode(Set ready, long nowMs) { + Set sendable = new HashSet(ready.size()); + for (Node node : ready) { + if (node == null) { + // we don't know about this topic/partition or it has no leader, re-fetch metadata + metadata.forceUpdate(); + } else if (nodeStates.isConnected(node.id()) && inFlightRequests.canSendMore(node.id())) { + sendable.add(node); + } else if (nodeStates.canConnect(node.id(), nowMs)) { + // we don't have a connection to this node right now, make one + initiateConnect(node, nowMs); + } + } + return sendable; + } + + /** + * Initiate a connection to the given node + */ + private void initiateConnect(Node node, long nowMs) { + try { + log.debug("Initiating connection to node {} at {}:{}.", node.id(), node.host(), node.port()); + selector.connect(node.id(), new InetSocketAddress(node.host(), node.port()), this.socketSendBuffer, this.socketReceiveBuffer); + this.nodeStates.connecting(node.id(), nowMs); + } catch (IOException e) { + /* attempt failed, we'll try again after the backoff */ + nodeStates.disconnected(node.id()); + /* maybe the problem is our metadata, update it */ + metadata.forceUpdate(); + log.debug("Error connecting to node {} at {}:{}:", node.id(), node.host(), node.port(), e); + } + } + + /** + * Handle any closed connections + */ + private void handleDisconnects(List disconnects, long nowMs) { + // clear out the in-flight requests for the disconnected broker + for (int node : disconnects) { + nodeStates.disconnected(node); + log.debug("Node {} disconnected.", node); + for (InFlightRequest request : this.inFlightRequests.clearAll(node)) { + log.trace("Cancelled request {} due to node {} being disconnected", request, node); + ApiKeys requestKey = ApiKeys.forId(request.request.header().apiKey()); + switch (requestKey) { + case PRODUCE: + int correlation = request.request.header().correlationId(); + for (RecordBatch batch : request.batches.values()) + completeBatch(batch, Errors.NETWORK_EXCEPTION, -1L, correlation, nowMs); + break; + case METADATA: + metadataFetchInProgress = false; + break; + default: + throw new IllegalArgumentException("Unexpected api key id: " + requestKey.id); + } + } + } + // we got a disconnect so we should probably refresh our metadata and see if that broker is dead + if (disconnects.size() > 0) + this.metadata.forceUpdate(); + } + + /** + * Record any connections that completed in our node state + */ + private void handleConnects(List connects) { + for (Integer id : connects) { + log.debug("Completed connection to node {}", id); + this.nodeStates.connected(id); + } + } + + /** + * Process completed sends + */ + public void handleSends(List sends) { + /* if acks = 0 then the request is satisfied once sent */ + for (NetworkSend send : sends) { + Deque requests = this.inFlightRequests.requestQueue(send.destination()); + InFlightRequest request = requests.peekFirst(); + log.trace("Completed send of request to node {}: {}", request.request.destination(), request.request); + if (!request.expectResponse) { + requests.pollFirst(); + if (request.request.header().apiKey() == ApiKeys.PRODUCE.id) { + for (RecordBatch batch : request.batches.values()) { + batch.done(-1L, Errors.NONE.exception()); + this.accumulator.deallocate(batch); + } + } + } + } + } + + /** + * Handle responses from the server + */ + private void handleResponses(List receives, long nowMs) { + for (NetworkReceive receive : receives) { + int source = receive.source(); + InFlightRequest req = inFlightRequests.nextCompleted(source); + ResponseHeader header = ResponseHeader.parse(receive.payload()); + short apiKey = req.request.header().apiKey(); + Struct body = (Struct) ProtoUtils.currentResponseSchema(apiKey).read(receive.payload()); + correlate(req.request.header(), header); + if (req.request.header().apiKey() == ApiKeys.PRODUCE.id) { + log.trace("Received produce response from node {} with correlation id {}", source, req.request.header().correlationId()); + handleProduceResponse(req, req.request.header(), body, nowMs); + } else if (req.request.header().apiKey() == ApiKeys.METADATA.id) { + log.trace("Received metadata response response from node {} with correlation id {}", source, req.request.header() + .correlationId()); + handleMetadataResponse(req.request.header(), body, nowMs); + } else { + throw new IllegalStateException("Unexpected response type: " + req.request.header().apiKey()); + } + this.sensors.recordLatency(receive.source(), nowMs - req.createdMs); + } + + } + + private void handleMetadataResponse(RequestHeader header, Struct body, long nowMs) { + this.metadataFetchInProgress = false; + MetadataResponse response = new MetadataResponse(body); + Cluster cluster = response.cluster(); + // 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) + this.metadata.update(cluster, nowMs); + else + log.trace("Ignoring empty metadata response with correlation id {}.", header.correlationId()); } /** * Handle a produce response */ - private void handleResponse(ClientResponse response, long now) { - int correlationId = response.request().request().header().correlationId(); - log.trace("Received produce response from node {} with correlation id {}", - response.request().request().destination(), - correlationId); - @SuppressWarnings("unchecked") - Map batches = (Map) response.request().attachment(); - // if we have a response, parse it - if (response.hasResponse()) { - ProduceResponse produceResponse = new ProduceResponse(response.responseBody()); - for (Map.Entry entry : produceResponse.responses().entrySet()) { + private void handleProduceResponse(InFlightRequest request, RequestHeader header, Struct body, long nowMs) { + ProduceResponse pr = new ProduceResponse(body); + for (Map responses : pr.responses().values()) { + for (Map.Entry entry : responses.entrySet()) { TopicPartition tp = entry.getKey(); - ProduceResponse.PartitionResponse partResp = entry.getValue(); - Errors error = Errors.forCode(partResp.errorCode); - RecordBatch batch = batches.get(tp); - completeBatch(batch, error, partResp.baseOffset, correlationId, now); + ProduceResponse.PartitionResponse response = entry.getValue(); + Errors error = Errors.forCode(response.errorCode); + if (error.exception() instanceof InvalidMetadataException) + metadata.forceUpdate(); + RecordBatch batch = request.batches.get(tp); + completeBatch(batch, error, response.baseOffset, header.correlationId(), nowMs); } - this.sensors.recordLatency(response.request().request().destination(), response.requestLatencyMs()); - } else { - // this is the acks = 0 case, just complete all requests - for (RecordBatch batch : batches.values()) - completeBatch(batch, Errors.NONE, -1L, correlationId, now); } } @@ -222,9 +466,9 @@ public class Sender implements Runnable { * @param error The error (or null if none) * @param baseOffset The base offset assigned to the records if successful * @param correlationId The correlation id for the request - * @param now The current POSIX time stamp in milliseconds + * @param nowMs The current POSIX time stamp in milliseconds */ - private void completeBatch(RecordBatch batch, Errors error, long baseOffset, long correlationId, long now) { + private void completeBatch(RecordBatch batch, Errors error, long baseOffset, long correlationId, long nowMs) { if (error != Errors.NONE && canRetry(batch, error)) { // retry log.warn("Got error produce response with correlation id {} on topic-partition {}, retrying ({} attempts left). Error: {}", @@ -232,7 +476,7 @@ public class Sender implements Runnable { batch.topicPartition, this.retries - batch.attempts - 1, error); - this.accumulator.reenqueue(batch, now); + this.accumulator.reenqueue(batch, nowMs); this.sensors.recordRetries(batch.topicPartition.topic(), batch.recordCount); } else { // tell the user the result of their request @@ -241,8 +485,6 @@ public class Sender implements Runnable { if (error != Errors.NONE) this.sensors.recordErrors(batch.topicPartition.topic(), batch.recordCount); } - if (error.exception() instanceof InvalidMetadataException) - metadata.forceUpdate(); } /** @@ -253,35 +495,257 @@ public class Sender implements Runnable { } /** + * Validate that the response corresponds to the request we expect or else explode + */ + private void correlate(RequestHeader requestHeader, ResponseHeader responseHeader) { + if (requestHeader.correlationId() != responseHeader.correlationId()) + throw new IllegalStateException("Correlation id for response (" + responseHeader.correlationId() + + ") does not match request (" + + requestHeader.correlationId() + + ")"); + } + + /** + * Create a metadata request for the given topics + */ + private InFlightRequest metadataRequest(long nowMs, int node, Set topics) { + MetadataRequest metadata = new MetadataRequest(new ArrayList(topics)); + RequestSend send = new RequestSend(node, header(ApiKeys.METADATA), metadata.toStruct()); + return new InFlightRequest(nowMs, true, send, null); + } + + /** * Transfer the record batches into a list of produce requests on a per-node basis */ - private List createProduceRequests(Map> collated, long now) { - List requests = new ArrayList(collated.size()); + private List generateProduceRequests(Map> collated, long nowMs) { + List requests = new ArrayList(collated.size()); for (Map.Entry> entry : collated.entrySet()) - requests.add(produceRequest(now, entry.getKey(), acks, requestTimeout, entry.getValue())); + requests.add(produceRequest(nowMs, entry.getKey(), acks, requestTimeout, entry.getValue())); return requests; } /** * Create a produce request from the given record batches */ - private ClientRequest produceRequest(long now, int destination, short acks, int timeout, List batches) { - ProduceRequest request = new ProduceRequest(acks, timeout); - Map recordsByPartition = new HashMap(batches.size()); + private InFlightRequest produceRequest(long nowMs, int destination, short acks, int timeout, List batches) { + Map batchesByPartition = new HashMap(); + Map> batchesByTopic = new HashMap>(); for (RecordBatch batch : batches) { - batch.records.buffer().flip(); - request.add(batch.topicPartition, batch.records); - recordsByPartition.put(batch.topicPartition, batch); + batchesByPartition.put(batch.topicPartition, batch); + List found = batchesByTopic.get(batch.topicPartition.topic()); + if (found == null) { + found = new ArrayList(); + batchesByTopic.put(batch.topicPartition.topic(), found); + } + found.add(batch); } - RequestSend send = new RequestSend(destination, this.client.nextRequestHeader(ApiKeys.PRODUCE), request.toStruct()); - return new ClientRequest(now, acks != 0, send, recordsByPartition); + Struct produce = new Struct(ProtoUtils.currentRequestSchema(ApiKeys.PRODUCE.id)); + produce.set("acks", acks); + produce.set("timeout", timeout); + List topicDatas = new ArrayList(batchesByTopic.size()); + for (Map.Entry> entry : batchesByTopic.entrySet()) { + Struct topicData = produce.instance("topic_data"); + topicData.set("topic", entry.getKey()); + List parts = entry.getValue(); + Object[] partitionData = new Object[parts.size()]; + for (int i = 0; i < parts.size(); i++) { + ByteBuffer buffer = parts.get(i).records.buffer(); + buffer.flip(); + Struct part = topicData.instance("data") + .set("partition", parts.get(i).topicPartition.partition()) + .set("record_set", buffer); + partitionData[i] = part; + } + topicData.set("data", partitionData); + topicDatas.add(topicData); + } + produce.set("topic_data", topicDatas.toArray()); + + RequestSend send = new RequestSend(destination, header(ApiKeys.PRODUCE), produce); + return new InFlightRequest(nowMs, acks != 0, send, batchesByPartition); + } + + private RequestHeader header(ApiKeys key) { + return new RequestHeader(key.id, clientId, correlation++); } /** * Wake up the selector associated with this send thread */ public void wakeup() { - this.client.wakeup(); + this.selector.wakeup(); + } + + /** + * The states of a node connection + */ + private static enum ConnectionState { + DISCONNECTED, CONNECTING, CONNECTED + } + + /** + * The state of a node + */ + private static final class NodeState { + private ConnectionState state; + private long lastConnectAttemptMs; + + public NodeState(ConnectionState state, long lastConnectAttempt) { + this.state = state; + this.lastConnectAttemptMs = lastConnectAttempt; + } + + public String toString() { + return "NodeState(" + state + ", " + lastConnectAttemptMs + ")"; + } + } + + private static class NodeStates { + private final long reconnectBackoffMs; + private final Map nodeState; + + public NodeStates(long reconnectBackoffMs) { + this.reconnectBackoffMs = reconnectBackoffMs; + this.nodeState = new HashMap(); + } + + public boolean canConnect(int node, long nowMs) { + NodeState state = nodeState.get(node); + if (state == null) + return true; + else + return state.state == ConnectionState.DISCONNECTED && nowMs - state.lastConnectAttemptMs > this.reconnectBackoffMs; + } + + public void connecting(int node, long nowMs) { + nodeState.put(node, new NodeState(ConnectionState.CONNECTING, nowMs)); + } + + public boolean isConnected(int node) { + NodeState state = nodeState.get(node); + return state != null && state.state == ConnectionState.CONNECTED; + } + + public boolean isConnecting(int node) { + NodeState state = nodeState.get(node); + return state != null && state.state == ConnectionState.CONNECTING; + } + + public void connected(int node) { + nodeState(node).state = ConnectionState.CONNECTED; + } + + public void disconnected(int node) { + nodeState(node).state = ConnectionState.DISCONNECTED; + } + + private NodeState nodeState(int node) { + NodeState state = this.nodeState.get(node); + if (state == null) + throw new IllegalStateException("No entry found for node " + node); + return state; + } + } + + /** + * An request that hasn't been fully processed yet + */ + private static final class InFlightRequest { + public long createdMs; + public boolean expectResponse; + public Map batches; + public RequestSend request; + + /** + * @param createdMs The unix timestamp in milliseonds for the time at which this request was created. + * @param expectResponse Should we expect a response message or is this request complete once it is sent? + * @param request The request + * @param batches The record batches contained in the request if it is a produce request + */ + public InFlightRequest(long createdMs, boolean expectResponse, RequestSend request, Map batches) { + this.createdMs = createdMs; + this.batches = batches; + this.request = request; + this.expectResponse = expectResponse; + } + + @Override + public String toString() { + return "InFlightRequest(expectResponse=" + expectResponse + ", batches=" + batches + ", request=" + request + ")"; + } + } + + /** + * A set of outstanding request queues for each node that have not yet received responses + */ + private static final class InFlightRequests { + private final int maxInFlightRequestsPerConnection; + private final Map> requests; + + public InFlightRequests(int maxInFlightRequestsPerConnection) { + this.requests = new HashMap>(); + this.maxInFlightRequestsPerConnection = maxInFlightRequestsPerConnection; + } + + /** + * Add the given request to the queue for the node it was directed to + */ + public void add(InFlightRequest request) { + Deque reqs = this.requests.get(request.request.destination()); + if (reqs == null) { + reqs = new ArrayDeque(); + this.requests.put(request.request.destination(), reqs); + } + reqs.addFirst(request); + } + + public Deque requestQueue(int node) { + Deque reqs = requests.get(node); + if (reqs == null || reqs.isEmpty()) + throw new IllegalStateException("Response from server for which there are no in-flight requests."); + return reqs; + } + + /** + * Get the oldest request (the one that that will be completed next) for the given node + */ + public InFlightRequest nextCompleted(int node) { + return requestQueue(node).pollLast(); + } + + /** + * Can we send more requests to this node? + * + * @param node Node in question + * @return true iff we have no requests still being sent to the given node + */ + public boolean canSendMore(int node) { + Deque queue = requests.get(node); + return queue == null || queue.isEmpty() || + (queue.peekFirst().request.complete() && queue.size() < this.maxInFlightRequestsPerConnection); + } + + /** + * Clear out all the in-flight requests for the given node and return them + * + * @param node The node + * @return All the in-flight requests for that node that have been removed + */ + public Iterable clearAll(int node) { + Deque reqs = requests.get(node); + if (reqs == null) { + return Collections.emptyList(); + } else { + return requests.remove(node); + } + } + + public int totalInFlightRequests() { + int total = 0; + for (Deque deque : this.requests.values()) + total += deque.size(); + return total; + } } /** @@ -296,7 +760,6 @@ public class Sender implements Runnable { public final Sensor requestTimeSensor; public final Sensor recordsPerRequestSensor; public final Sensor batchSizeSensor; - public final Sensor compressionRateSensor; public final Sensor maxRecordSizeSensor; public SenderMetrics(Metrics metrics) { @@ -305,9 +768,6 @@ public class Sender implements Runnable { this.batchSizeSensor = metrics.sensor("batch-size"); this.batchSizeSensor.add("batch-size-avg", "The average number of bytes sent per partition per-request.", new Avg()); - this.compressionRateSensor = metrics.sensor("compression-rate"); - this.compressionRateSensor.add("compression-rate-avg", "The average compression rate of record batches.", new Avg()); - this.queueTimeSensor = metrics.sensor("queue-time"); this.queueTimeSensor.add("record-queue-time-avg", "The average time in ms record batches spent in the record accumulator.", @@ -334,13 +794,13 @@ public class Sender implements Runnable { this.maxRecordSizeSensor.add("record-size-max", "The maximum record size", new Max()); this.metrics.addMetric("requests-in-flight", "The current number of in-flight requests awaiting a response.", new Measurable() { - public double measure(MetricConfig config, long now) { - return client.inFlightRequestCount(); + public double measure(MetricConfig config, long nowMs) { + return inFlightRequests.totalInFlightRequests(); } }); metrics.addMetric("metadata-age", "The age in seconds of the current producer metadata being used.", new Measurable() { - public double measure(MetricConfig config, long now) { - return (now - metadata.lastUpdate()) / 1000.0; + public double measure(MetricConfig config, long nowMs) { + return (nowMs - metadata.lastUpdate()) / 1000.0; } }); } @@ -358,10 +818,6 @@ public class Sender implements Runnable { Sensor topicByteRate = this.metrics.sensor(topicByteRateName); topicByteRate.add("topic." + topic + ".byte-rate", new Rate()); - String topicCompressionRateName = "topic." + topic + ".compression-rate"; - Sensor topicCompressionRate = this.metrics.sensor(topicCompressionRateName); - topicCompressionRate.add("topic." + topic + ".compression-rate", new Avg()); - String topicRetryName = "topic." + topic + ".record-retries"; Sensor topicRetrySensor = this.metrics.sensor(topicRetryName); topicRetrySensor.add("topic." + topic + ".record-retry-rate", new Rate()); @@ -372,15 +828,14 @@ public class Sender implements Runnable { } } - public void updateProduceRequestMetrics(List requests) { - long now = time.milliseconds(); + public void updateProduceRequestMetrics(List requests) { + long nowMs = time.milliseconds(); for (int i = 0; i < requests.size(); i++) { - ClientRequest request = requests.get(i); + InFlightRequest request = requests.get(i); int records = 0; - if (request.attachment() != null) { - Map responseBatches = (Map) request.attachment(); - for (RecordBatch batch : responseBatches.values()) { + if (request.batches != null) { + for (RecordBatch batch : request.batches.values()) { // register all per-topic metrics at once String topic = batch.topicPartition.topic(); @@ -396,49 +851,43 @@ public class Sender implements Runnable { Sensor topicByteRate = Utils.notNull(this.metrics.getSensor(topicByteRateName)); topicByteRate.record(batch.records.sizeInBytes()); - // per-topic compression rate - String topicCompressionRateName = "topic." + topic + ".compression-rate"; - Sensor topicCompressionRate = Utils.notNull(this.metrics.getSensor(topicCompressionRateName)); - topicCompressionRate.record(batch.records.compressionRate()); - // global metrics - this.batchSizeSensor.record(batch.records.sizeInBytes(), now); - this.queueTimeSensor.record(batch.drainedMs - batch.createdMs, now); - this.compressionRateSensor.record(batch.records.compressionRate()); - this.maxRecordSizeSensor.record(batch.maxRecordSize, now); + this.batchSizeSensor.record(batch.records.sizeInBytes(), nowMs); + this.queueTimeSensor.record(batch.drainedMs - batch.createdMs, nowMs); + this.maxRecordSizeSensor.record(batch.maxRecordSize, nowMs); records += batch.recordCount; } - this.recordsPerRequestSensor.record(records, now); + this.recordsPerRequestSensor.record(records, nowMs); } } } public void recordRetries(String topic, int count) { - long now = time.milliseconds(); - this.retrySensor.record(count, now); + long nowMs = time.milliseconds(); + this.retrySensor.record(count, nowMs); String topicRetryName = "topic." + topic + ".record-retries"; Sensor topicRetrySensor = this.metrics.getSensor(topicRetryName); if (topicRetrySensor != null) - topicRetrySensor.record(count, now); + topicRetrySensor.record(count, nowMs); } public void recordErrors(String topic, int count) { - long now = time.milliseconds(); - this.errorSensor.record(count, now); + long nowMs = time.milliseconds(); + this.errorSensor.record(count, nowMs); String topicErrorName = "topic." + topic + ".record-errors"; Sensor topicErrorSensor = this.metrics.getSensor(topicErrorName); - if (topicErrorSensor != null) - topicErrorSensor.record(count, now); + if (topicErrorSensor != null) + topicErrorSensor.record(count, nowMs); } public void recordLatency(int node, long latency) { - long now = time.milliseconds(); - this.requestTimeSensor.record(latency, now); + long nowMs = time.milliseconds(); + this.requestTimeSensor.record(latency, nowMs); if (node >= 0) { String nodeTimeName = "node-" + node + ".latency"; Sensor nodeRequestTime = this.metrics.getSensor(nodeTimeName); if (nodeRequestTime != null) - nodeRequestTime.record(latency, now); + nodeRequestTime.record(latency, nowMs); } } } diff --git a/clients/src/main/java/org/apache/kafka/common/metrics/Measurable.java b/clients/src/main/java/org/apache/kafka/common/metrics/Measurable.java index 79f61bc..7c2e33c 100644 --- a/clients/src/main/java/org/apache/kafka/common/metrics/Measurable.java +++ b/clients/src/main/java/org/apache/kafka/common/metrics/Measurable.java @@ -1,14 +1,18 @@ /** - * 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. + * 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.metrics; @@ -20,9 +24,9 @@ public interface Measurable { /** * Measure this quantity and return the result as a double * @param config The configuration for this metric - * @param now The POSIX time in milliseconds the measurement is being taken + * @param nowMs The POSIX time in milliseconds the measurement is being taken * @return The measured value */ - public double measure(MetricConfig config, long now); + public double measure(MetricConfig config, long nowMs); } diff --git a/clients/src/main/java/org/apache/kafka/common/metrics/stats/Avg.java b/clients/src/main/java/org/apache/kafka/common/metrics/stats/Avg.java index ed6767f..c9963cb 100644 --- a/clients/src/main/java/org/apache/kafka/common/metrics/stats/Avg.java +++ b/clients/src/main/java/org/apache/kafka/common/metrics/stats/Avg.java @@ -1,14 +1,18 @@ /** - * 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. + * 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.metrics.stats; @@ -16,6 +20,7 @@ import java.util.List; import org.apache.kafka.common.metrics.MetricConfig; + /** * A {@link SampledStat} that maintains a simple average over its samples. */ @@ -26,12 +31,12 @@ public class Avg extends SampledStat { } @Override - protected void update(Sample sample, MetricConfig config, double value, long now) { + protected void update(Sample sample, MetricConfig config, double value, long timeMs) { sample.value += value; } @Override - public double combine(List samples, MetricConfig config, long now) { + public double combine(List samples, MetricConfig config, long nowMs) { double total = 0.0; long count = 0; for (int i = 0; i < samples.size(); i++) { diff --git a/clients/src/main/java/org/apache/kafka/common/metrics/stats/Count.java b/clients/src/main/java/org/apache/kafka/common/metrics/stats/Count.java index 90c0bf5..efcd61b 100644 --- a/clients/src/main/java/org/apache/kafka/common/metrics/stats/Count.java +++ b/clients/src/main/java/org/apache/kafka/common/metrics/stats/Count.java @@ -1,14 +1,18 @@ /** - * 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. + * 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.metrics.stats; @@ -16,6 +20,7 @@ import java.util.List; import org.apache.kafka.common.metrics.MetricConfig; + /** * A {@link SampledStat} that maintains a simple count of what it has seen. */ @@ -26,12 +31,12 @@ public class Count extends SampledStat { } @Override - protected void update(Sample sample, MetricConfig config, double value, long now) { + protected void update(Sample sample, MetricConfig config, double value, long timeMs) { sample.value += 1.0; } @Override - public double combine(List samples, MetricConfig config, long now) { + public double combine(List samples, MetricConfig config, long nowMs) { double total = 0.0; for (int i = 0; i < samples.size(); i++) total += samples.get(i).value; diff --git a/clients/src/main/java/org/apache/kafka/common/metrics/stats/Max.java b/clients/src/main/java/org/apache/kafka/common/metrics/stats/Max.java index 6bbb0a3..c492c38 100644 --- a/clients/src/main/java/org/apache/kafka/common/metrics/stats/Max.java +++ b/clients/src/main/java/org/apache/kafka/common/metrics/stats/Max.java @@ -1,14 +1,18 @@ /** - * 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. + * 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.metrics.stats; @@ -16,6 +20,7 @@ import java.util.List; import org.apache.kafka.common.metrics.MetricConfig; + /** * A {@link SampledStat} that gives the max over its samples. */ @@ -26,12 +31,12 @@ public final class Max extends SampledStat { } @Override - protected void update(Sample sample, MetricConfig config, double value, long now) { + protected void update(Sample sample, MetricConfig config, double value, long timeMs) { sample.value = Math.max(sample.value, value); } @Override - public double combine(List samples, MetricConfig config, long now) { + public double combine(List samples, MetricConfig config, long nowMs) { double max = Double.NEGATIVE_INFINITY; for (int i = 0; i < samples.size(); i++) max = Math.max(max, samples.get(i).value); diff --git a/clients/src/main/java/org/apache/kafka/common/metrics/stats/Min.java b/clients/src/main/java/org/apache/kafka/common/metrics/stats/Min.java index 9f74417..bd0919c 100644 --- a/clients/src/main/java/org/apache/kafka/common/metrics/stats/Min.java +++ b/clients/src/main/java/org/apache/kafka/common/metrics/stats/Min.java @@ -1,14 +1,18 @@ /** - * 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. + * 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.metrics.stats; @@ -16,6 +20,7 @@ import java.util.List; import org.apache.kafka.common.metrics.MetricConfig; + /** * A {@link SampledStat} that gives the min over its samples. */ @@ -26,12 +31,12 @@ public class Min extends SampledStat { } @Override - protected void update(Sample sample, MetricConfig config, double value, long now) { + protected void update(Sample sample, MetricConfig config, double value, long timeMs) { sample.value = Math.min(sample.value, value); } @Override - public double combine(List samples, MetricConfig config, long now) { + public double combine(List samples, MetricConfig config, long nowMs) { double max = Double.MAX_VALUE; for (int i = 0; i < samples.size(); i++) max = Math.min(max, samples.get(i).value); diff --git a/clients/src/main/java/org/apache/kafka/common/metrics/stats/Percentiles.java b/clients/src/main/java/org/apache/kafka/common/metrics/stats/Percentiles.java index c70d577..8300978 100644 --- a/clients/src/main/java/org/apache/kafka/common/metrics/stats/Percentiles.java +++ b/clients/src/main/java/org/apache/kafka/common/metrics/stats/Percentiles.java @@ -1,14 +1,18 @@ /** - * 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. + * 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.metrics.stats; @@ -22,6 +26,7 @@ import org.apache.kafka.common.metrics.stats.Histogram.BinScheme; import org.apache.kafka.common.metrics.stats.Histogram.ConstantBinScheme; import org.apache.kafka.common.metrics.stats.Histogram.LinearBinScheme; + /** * A compound stat that reports one or more percentiles */ @@ -60,16 +65,16 @@ public class Percentiles extends SampledStat implements CompoundStat { for (Percentile percentile : this.percentiles) { final double pct = percentile.percentile(); ms.add(new NamedMeasurable(percentile.name(), percentile.description(), new Measurable() { - public double measure(MetricConfig config, long now) { - return value(config, now, pct / 100.0); + public double measure(MetricConfig config, long nowMs) { + return value(config, nowMs, pct / 100.0); } })); } return ms; } - public double value(MetricConfig config, long now, double quantile) { - purgeObsoleteSamples(config, now); + public double value(MetricConfig config, long nowMs, double quantile) { + purgeObsoleteSamples(config, nowMs); float count = 0.0f; for (Sample sample : this.samples) count += sample.eventCount; @@ -89,8 +94,8 @@ public class Percentiles extends SampledStat implements CompoundStat { return Double.POSITIVE_INFINITY; } - public double combine(List samples, MetricConfig config, long now) { - return value(config, now, 0.5); + public double combine(List samples, MetricConfig config, long nowMs) { + return value(config, nowMs, 0.5); } @Override diff --git a/clients/src/main/java/org/apache/kafka/common/metrics/stats/Rate.java b/clients/src/main/java/org/apache/kafka/common/metrics/stats/Rate.java index a5838b3..4b481a5 100644 --- a/clients/src/main/java/org/apache/kafka/common/metrics/stats/Rate.java +++ b/clients/src/main/java/org/apache/kafka/common/metrics/stats/Rate.java @@ -56,9 +56,9 @@ public class Rate implements MeasurableStat { } @Override - public double measure(MetricConfig config, long now) { - double value = stat.measure(config, now); - double elapsed = convert(now - stat.oldest(now).lastWindowMs); + public double measure(MetricConfig config, long nowMs) { + double value = stat.measure(config, nowMs); + double elapsed = convert(nowMs - stat.oldest(nowMs).lastWindowMs); return value / elapsed; } @@ -95,7 +95,7 @@ public class Rate implements MeasurableStat { } @Override - public double combine(List samples, MetricConfig config, long now) { + public double combine(List samples, MetricConfig config, long nowMs) { double total = 0.0; for (int i = 0; i < samples.size(); i++) total += samples.get(i).value; diff --git a/clients/src/main/java/org/apache/kafka/common/metrics/stats/SampledStat.java b/clients/src/main/java/org/apache/kafka/common/metrics/stats/SampledStat.java index b341b7d..0d4056f 100644 --- a/clients/src/main/java/org/apache/kafka/common/metrics/stats/SampledStat.java +++ b/clients/src/main/java/org/apache/kafka/common/metrics/stats/SampledStat.java @@ -66,9 +66,9 @@ public abstract class SampledStat implements MeasurableStat { } @Override - public double measure(MetricConfig config, long now) { - purgeObsoleteSamples(config, now); - return combine(this.samples, config, now); + public double measure(MetricConfig config, long nowMs) { + purgeObsoleteSamples(config, nowMs); + return combine(this.samples, config, nowMs); } public Sample current(long timeMs) { @@ -77,9 +77,9 @@ public abstract class SampledStat implements MeasurableStat { return this.samples.get(this.current); } - public Sample oldest(long now) { + public Sample oldest(long nowMs) { if (samples.size() == 0) - this.samples.add(newSample(now)); + this.samples.add(newSample(nowMs)); Sample oldest = this.samples.get(0); for (int i = 1; i < this.samples.size(); i++) { Sample curr = this.samples.get(i); @@ -91,15 +91,15 @@ public abstract class SampledStat implements MeasurableStat { protected abstract void update(Sample sample, MetricConfig config, double value, long timeMs); - public abstract double combine(List samples, MetricConfig config, long now); + public abstract double combine(List samples, MetricConfig config, long nowMs); /* Timeout any windows that have expired in the absence of any events */ - protected void purgeObsoleteSamples(MetricConfig config, long now) { + protected void purgeObsoleteSamples(MetricConfig config, long nowMs) { long expireAge = config.samples() * config.timeWindowMs(); for (int i = 0; i < samples.size(); i++) { Sample sample = this.samples.get(i); - if (now - sample.lastWindowMs >= expireAge) - sample.reset(now); + if (nowMs - sample.lastWindowMs >= expireAge) + sample.reset(nowMs); } } diff --git a/clients/src/main/java/org/apache/kafka/common/metrics/stats/Total.java b/clients/src/main/java/org/apache/kafka/common/metrics/stats/Total.java index 67999a9..53dd3d5 100644 --- a/clients/src/main/java/org/apache/kafka/common/metrics/stats/Total.java +++ b/clients/src/main/java/org/apache/kafka/common/metrics/stats/Total.java @@ -1,14 +1,18 @@ /** - * 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. + * 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.metrics.stats; @@ -31,12 +35,12 @@ public class Total implements MeasurableStat { } @Override - public void record(MetricConfig config, double value, long now) { + public void record(MetricConfig config, double value, long timeMs) { this.total += value; } @Override - public double measure(MetricConfig config, long now) { + public double measure(MetricConfig config, long nowMs) { return this.total; } diff --git a/clients/src/main/java/org/apache/kafka/common/network/ByteBufferSend.java b/clients/src/main/java/org/apache/kafka/common/network/ByteBufferSend.java index c8213e1..6350424 100644 --- a/clients/src/main/java/org/apache/kafka/common/network/ByteBufferSend.java +++ b/clients/src/main/java/org/apache/kafka/common/network/ByteBufferSend.java @@ -42,7 +42,7 @@ public class ByteBufferSend implements Send { } @Override - public boolean completed() { + public boolean complete() { return remaining <= 0; } diff --git a/clients/src/main/java/org/apache/kafka/common/network/Selector.java b/clients/src/main/java/org/apache/kafka/common/network/Selector.java index 93f2f1c..3e35898 100644 --- a/clients/src/main/java/org/apache/kafka/common/network/Selector.java +++ b/clients/src/main/java/org/apache/kafka/common/network/Selector.java @@ -414,8 +414,8 @@ public class Selector implements Selectable { this.bytesTransferred = this.metrics.sensor("bytes-sent-received"); bytesTransferred.add("network-io-rate", - "The average number of network operations (reads or writes) on all connections per second.", - new Rate(new Count())); + "The average number of network operations (reads or writes) on all connections per second.", + new Rate(new Count())); this.bytesSent = this.metrics.sensor("bytes-sent", bytesTransferred); this.bytesSent.add("outgoing-byte-rate", "The average number of outgoing bytes sent per second to all servers.", new Rate()); @@ -429,11 +429,11 @@ public class Selector implements Selectable { this.selectTime = this.metrics.sensor("select-time"); this.selectTime.add("select-rate", - "Number of times the I/O layer checked for new I/O to perform per second", - new Rate(new Count())); + "Number of times the I/O layer checked for new I/O to perform per second", + new Rate(new Count())); this.selectTime.add("io-wait-time-ns-avg", - "The average length of time the I/O thread spent waiting for a socket ready for reads or writes in nanoseconds.", - new Avg()); + "The average length of time the I/O thread spent waiting for a socket ready for reads or writes in nanoseconds.", + new Avg()); this.selectTime.add("io-wait-ratio", "The fraction of time the I/O thread spent waiting.", new Rate(TimeUnit.NANOSECONDS)); this.ioTime = this.metrics.sensor("io-time"); @@ -441,7 +441,7 @@ public class Selector implements Selectable { this.ioTime.add("io-ratio", "The fraction of time the I/O thread spent doing I/O", new Rate(TimeUnit.NANOSECONDS)); this.metrics.addMetric("connection-count", "The current number of active connections.", new Measurable() { - public double measure(MetricConfig config, long now) { + public double measure(MetricConfig config, long nowMs) { return keys.size(); } }); @@ -456,9 +456,7 @@ public class Selector implements Selectable { if (nodeRequest == null) { nodeRequest = this.metrics.sensor(nodeRequestName); nodeRequest.add("node-" + node + ".outgoing-byte-rate", new Rate()); - nodeRequest.add("node-" + node + ".request-rate", - "The average number of requests sent per second.", - new Rate(new Count())); + nodeRequest.add("node-" + node + ".request-rate", "The average number of requests sent per second.", new Rate(new Count())); nodeRequest.add("node-" + node + ".request-size-avg", "The average size of all requests in the window..", new Avg()); nodeRequest.add("node-" + node + ".request-size-max", "The maximum size of any request sent in the window.", new Max()); @@ -466,8 +464,8 @@ public class Selector implements Selectable { Sensor nodeResponse = this.metrics.sensor(nodeResponseName); nodeResponse.add("node-" + node + ".incoming-byte-rate", new Rate()); nodeResponse.add("node-" + node + ".response-rate", - "The average number of responses received per second.", - new Rate(new Count())); + "The average number of responses received per second.", + new Rate(new Count())); String nodeTimeName = "node-" + node + ".latency"; Sensor nodeRequestTime = this.metrics.sensor(nodeTimeName); @@ -478,24 +476,22 @@ public class Selector implements Selectable { } public void recordBytesSent(int node, int bytes) { - long now = time.milliseconds(); - this.bytesSent.record(bytes, now); + long nowMs = time.milliseconds(); + this.bytesSent.record(bytes, nowMs); if (node >= 0) { String nodeRequestName = "node-" + node + ".bytes-sent"; Sensor nodeRequest = this.metrics.getSensor(nodeRequestName); - if (nodeRequest != null) - nodeRequest.record(bytes, now); + if (nodeRequest != null) nodeRequest.record(bytes, nowMs); } } public void recordBytesReceived(int node, int bytes) { - long now = time.milliseconds(); - this.bytesReceived.record(bytes, now); + long nowMs = time.milliseconds(); + this.bytesReceived.record(bytes, nowMs); if (node >= 0) { String nodeRequestName = "node-" + node + ".bytes-received"; Sensor nodeRequest = this.metrics.getSensor(nodeRequestName); - if (nodeRequest != null) - nodeRequest.record(bytes, now); + if (nodeRequest != null) nodeRequest.record(bytes, nowMs); } } } diff --git a/clients/src/main/java/org/apache/kafka/common/network/Send.java b/clients/src/main/java/org/apache/kafka/common/network/Send.java index 5d321a0..d62dff9 100644 --- a/clients/src/main/java/org/apache/kafka/common/network/Send.java +++ b/clients/src/main/java/org/apache/kafka/common/network/Send.java @@ -1,14 +1,18 @@ /** - * 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. + * 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; @@ -34,7 +38,7 @@ public interface Send { /** * Is this send complete? */ - public boolean completed(); + public boolean complete(); /** * An optional method to turn this send into an array of ByteBuffers if possible (otherwise returns null) diff --git a/clients/src/main/java/org/apache/kafka/common/protocol/types/Schema.java b/clients/src/main/java/org/apache/kafka/common/protocol/types/Schema.java index 7164701..68b8827 100644 --- a/clients/src/main/java/org/apache/kafka/common/protocol/types/Schema.java +++ b/clients/src/main/java/org/apache/kafka/common/protocol/types/Schema.java @@ -1,14 +1,18 @@ /** - * 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. + * 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.protocol.types; @@ -47,9 +51,8 @@ public class Schema extends Type { Object value = f.type().validate(r.get(f)); f.type.write(buffer, value); } catch (Exception e) { - throw new SchemaException("Error writing field '" + f.name + - "': " + - (e.getMessage() == null ? e.getClass().getName() : e.getMessage())); + throw new SchemaException("Error writing field '" + f.name + "': " + e.getMessage() == null ? e.getMessage() : e.getClass() + .getName()); } } } @@ -63,9 +66,8 @@ public class Schema extends Type { try { objects[i] = fields[i].type.read(buffer); } catch (Exception e) { - throw new SchemaException("Error reading field '" + fields[i].name + - "': " + - (e.getMessage() == null ? e.getClass().getName() : e.getMessage())); + throw new SchemaException("Error reading field '" + fields[i].name + "': " + e.getMessage() == null ? e.getMessage() + : e.getClass().getName()); } } return new Struct(this, objects); diff --git a/clients/src/main/java/org/apache/kafka/common/record/Compressor.java b/clients/src/main/java/org/apache/kafka/common/record/Compressor.java index 0323f5f..0fa6dd2 100644 --- a/clients/src/main/java/org/apache/kafka/common/record/Compressor.java +++ b/clients/src/main/java/org/apache/kafka/common/record/Compressor.java @@ -80,14 +80,6 @@ public class Compressor { public ByteBuffer buffer() { return bufferStream.buffer(); } - - public double compressionRate() { - ByteBuffer buffer = bufferStream.buffer(); - if (this.writtenUncompressed == 0) - return 1.0; - else - return (double) buffer.position() / this.writtenUncompressed; - } public void close() { try { diff --git a/clients/src/main/java/org/apache/kafka/common/record/MemoryRecords.java b/clients/src/main/java/org/apache/kafka/common/record/MemoryRecords.java index 759f577..428968c 100644 --- a/clients/src/main/java/org/apache/kafka/common/record/MemoryRecords.java +++ b/clients/src/main/java/org/apache/kafka/common/record/MemoryRecords.java @@ -1,14 +1,18 @@ /** - * 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. + * 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.record; @@ -84,14 +88,14 @@ public class MemoryRecords implements Records { /** * Check if we have room for a new record containing the given key/value pair - * - * Note that the return value is based on the estimate of the bytes written to the compressor, which may not be - * accurate if compression is really used. When this happens, the following append may cause dynamic buffer - * re-allocation in the underlying byte buffer stream. + * + * Note that the return value is based on the estimate of the bytes written to the compressor, + * which may not be accurate if compression is really used. When this happens, the following + * append may cause dynamic buffer re-allocation in the underlying byte buffer stream. */ public boolean hasRoomFor(byte[] key, byte[] value) { - return this.writable && this.capacity >= this.compressor.estimatedBytesWritten() + Records.LOG_OVERHEAD + - Record.recordSize(key, value); + return this.writable && + this.capacity >= this.compressor.estimatedBytesWritten() + Records.LOG_OVERHEAD + Record.recordSize(key, value); } public boolean isFull() { @@ -118,16 +122,6 @@ public class MemoryRecords implements Records { public int sizeInBytes() { return compressor.buffer().position(); } - - /** - * The compression rate of this record set - */ - public double compressionRate() { - if (compressor == null) - return 1.0; - else - return compressor.compressionRate(); - } /** * Return the capacity of the buffer @@ -165,10 +159,10 @@ public class MemoryRecords implements Records { /* * Read the next record from the buffer. - * - * Note that in the compressed message set, each message value size is set as the size of the un-compressed - * version of the message value, so when we do de-compression allocating an array of the specified size for - * reading compressed value data is sufficient. + * + * Note that in the compressed message set, each message value size is set as the size + * of the un-compressed version of the message value, so when we do de-compression + * allocating an array of the specified size for reading compressed value data is sufficient. */ @Override protected LogEntry makeNext() { diff --git a/clients/src/main/java/org/apache/kafka/common/requests/ProduceRequest.java b/clients/src/main/java/org/apache/kafka/common/requests/ProduceRequest.java deleted file mode 100644 index 6036f6a..0000000 --- a/clients/src/main/java/org/apache/kafka/common/requests/ProduceRequest.java +++ /dev/null @@ -1,71 +0,0 @@ -package org.apache.kafka.common.requests; - -import java.nio.ByteBuffer; -import java.util.ArrayList; -import java.util.HashMap; -import java.util.List; -import java.util.Map; - -import org.apache.kafka.common.TopicPartition; -import org.apache.kafka.common.protocol.ApiKeys; -import org.apache.kafka.common.protocol.ProtoUtils; -import org.apache.kafka.common.protocol.types.Struct; -import org.apache.kafka.common.record.MemoryRecords; - -public class ProduceRequest { - - private final short acks; - private final int timeout; - private final Map> records; - - public ProduceRequest(short acks, int timeout) { - this.acks = acks; - this.timeout = timeout; - this.records = new HashMap>(); - } - - public void add(TopicPartition tp, MemoryRecords recs) { - List found = this.records.get(tp.topic()); - if (found == null) { - found = new ArrayList(); - records.put(tp.topic(), found); - } - found.add(new PartitionRecords(tp, recs)); - } - - public Struct toStruct() { - Struct produce = new Struct(ProtoUtils.currentRequestSchema(ApiKeys.PRODUCE.id)); - produce.set("acks", acks); - produce.set("timeout", timeout); - List topicDatas = new ArrayList(records.size()); - for (Map.Entry> entry : records.entrySet()) { - Struct topicData = produce.instance("topic_data"); - topicData.set("topic", entry.getKey()); - List parts = entry.getValue(); - Object[] partitionData = new Object[parts.size()]; - for (int i = 0; i < parts.size(); i++) { - ByteBuffer buffer = parts.get(i).records.buffer(); - buffer.flip(); - Struct part = topicData.instance("data") - .set("partition", parts.get(i).topicPartition.partition()) - .set("record_set", buffer); - partitionData[i] = part; - } - topicData.set("data", partitionData); - topicDatas.add(topicData); - } - produce.set("topic_data", topicDatas.toArray()); - return produce; - } - - private static final class PartitionRecords { - public final TopicPartition topicPartition; - public final MemoryRecords records; - - public PartitionRecords(TopicPartition topicPartition, MemoryRecords records) { - this.topicPartition = topicPartition; - this.records = records; - } - } - -} diff --git a/clients/src/main/java/org/apache/kafka/common/requests/ProduceResponse.java b/clients/src/main/java/org/apache/kafka/common/requests/ProduceResponse.java index 6cf4fb7..6fa4a58 100644 --- a/clients/src/main/java/org/apache/kafka/common/requests/ProduceResponse.java +++ b/clients/src/main/java/org/apache/kafka/common/requests/ProduceResponse.java @@ -3,50 +3,65 @@ * 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 java.util.HashMap; -import java.util.Map; - import org.apache.kafka.common.TopicPartition; import org.apache.kafka.common.protocol.types.Struct; -public class ProduceResponse { +import java.util.HashMap; +import java.util.Map; - private final Map responses; +public class ProduceResponse { + public class PartitionResponse { + public int partitionId; + public short errorCode; + public long baseOffset; - public ProduceResponse() { - this.responses = new HashMap(); + public PartitionResponse(int partitionId, short errorCode, long baseOffset) { + this.partitionId = partitionId; + this.errorCode = errorCode; + this.baseOffset = baseOffset; + } + @Override + public String toString() { + StringBuilder b = new StringBuilder(); + b.append('{'); + b.append("pid: " + partitionId); + b.append(",error: " + errorCode); + b.append(",offset: " + baseOffset); + b.append('}'); + return b.toString(); + } } + private final Map> responses; + public ProduceResponse(Struct struct) { - responses = new HashMap(); + responses = new HashMap>(); for (Object topicResponse : (Object[]) struct.get("responses")) { Struct topicRespStruct = (Struct) topicResponse; String topic = (String) topicRespStruct.get("topic"); + Map topicResponses = new HashMap(); for (Object partResponse : (Object[]) topicRespStruct.get("partition_responses")) { Struct partRespStruct = (Struct) partResponse; int partition = (Integer) partRespStruct.get("partition"); short errorCode = (Short) partRespStruct.get("error_code"); long offset = (Long) partRespStruct.get("base_offset"); TopicPartition tp = new TopicPartition(topic, partition); - responses.put(tp, new PartitionResponse(partition, errorCode, offset)); + topicResponses.put(tp, new PartitionResponse(partition, errorCode, offset)); } + responses.put(topic, topicResponses); } } - public void addResponse(TopicPartition tp, int partition, short error, long baseOffset) { - this.responses.put(tp, new PartitionResponse(partition, error, baseOffset)); - } - - public Map responses() { + public Map> responses() { return this.responses; } @@ -55,40 +70,16 @@ public class ProduceResponse { StringBuilder b = new StringBuilder(); b.append('{'); boolean isFirst = true; - for (Map.Entry entry : responses.entrySet()) { - if (isFirst) - isFirst = false; - else - b.append(','); - b.append(entry.getKey() + " : " + entry.getValue()); + for (Map response : responses.values()) { + for (Map.Entry entry : response.entrySet()) { + if (isFirst) + isFirst = false; + else + b.append(','); + b.append(entry.getKey() + " : " + entry.getValue()); + } } b.append('}'); return b.toString(); } - - public static class PartitionResponse { - public int partitionId; - public short errorCode; - public long baseOffset; - - public PartitionResponse(int partitionId, short errorCode, long baseOffset) { - this.partitionId = partitionId; - this.errorCode = errorCode; - this.baseOffset = baseOffset; - } - - @Override - public String toString() { - StringBuilder b = new StringBuilder(); - b.append('{'); - b.append("pid: "); - b.append(partitionId); - b.append(",error: "); - b.append(errorCode); - b.append(",offset: "); - b.append(baseOffset); - b.append('}'); - return b.toString(); - } - } } diff --git a/clients/src/test/java/org/apache/kafka/clients/MockClient.java b/clients/src/test/java/org/apache/kafka/clients/MockClient.java deleted file mode 100644 index aae8d4a..0000000 --- a/clients/src/test/java/org/apache/kafka/clients/MockClient.java +++ /dev/null @@ -1,96 +0,0 @@ -package org.apache.kafka.clients; - -import java.util.ArrayDeque; -import java.util.ArrayList; -import java.util.HashSet; -import java.util.Iterator; -import java.util.List; -import java.util.Queue; -import java.util.Set; - -import org.apache.kafka.common.Node; -import org.apache.kafka.common.protocol.ApiKeys; -import org.apache.kafka.common.protocol.types.Struct; -import org.apache.kafka.common.requests.RequestHeader; -import org.apache.kafka.common.utils.Time; - -/** - * A mock network client for use testing code - */ -public class MockClient implements KafkaClient { - - private final Time time; - private int correlation = 0; - private final Set ready = new HashSet(); - private final Queue requests = new ArrayDeque(); - private final Queue responses = new ArrayDeque(); - - public MockClient(Time time) { - this.time = time; - } - - @Override - public boolean isReady(Node node, long now) { - return ready.contains(node.id()); - } - - @Override - public boolean ready(Node node, long now) { - boolean found = isReady(node, now); - ready.add(node.id()); - return found; - } - - public void disconnect(Integer node) { - Iterator iter = requests.iterator(); - while (iter.hasNext()) { - ClientRequest request = iter.next(); - if (request.request().destination() == node) { - responses.add(new ClientResponse(request, time.milliseconds(), true, null)); - iter.remove(); - } - } - ready.remove(node); - } - - @Override - public List poll(List requests, long timeoutMs, long now) { - this.requests.addAll(requests); - List copy = new ArrayList(this.responses); - this.responses.clear(); - return copy; - } - - public Queue requests() { - return this.requests; - } - - public void respond(Struct body) { - ClientRequest request = requests.remove(); - responses.add(new ClientResponse(request, time.milliseconds(), false, body)); - } - - @Override - public int inFlightRequestCount() { - return requests.size(); - } - - @Override - public RequestHeader nextRequestHeader(ApiKeys key) { - return new RequestHeader(key.id, "mock", correlation++); - } - - @Override - public void wakeup() { - } - - @Override - public void close() { - } - - @Override - public Node leastLoadedNode(long now) { - return null; - } - -} diff --git a/clients/src/test/java/org/apache/kafka/clients/NetworkClientTest.java b/clients/src/test/java/org/apache/kafka/clients/NetworkClientTest.java deleted file mode 100644 index 6a3cdcc..0000000 --- a/clients/src/test/java/org/apache/kafka/clients/NetworkClientTest.java +++ /dev/null @@ -1,99 +0,0 @@ -package org.apache.kafka.clients; - -import static org.junit.Assert.assertEquals; -import static org.junit.Assert.assertFalse; -import static org.junit.Assert.assertTrue; - -import java.nio.ByteBuffer; -import java.util.ArrayList; -import java.util.Arrays; -import java.util.List; - -import org.apache.kafka.clients.producer.internals.Metadata; -import org.apache.kafka.common.Cluster; -import org.apache.kafka.common.Node; -import org.apache.kafka.common.network.NetworkReceive; -import org.apache.kafka.common.protocol.ApiKeys; -import org.apache.kafka.common.protocol.ProtoUtils; -import org.apache.kafka.common.protocol.types.Struct; -import org.apache.kafka.common.requests.MetadataRequest; -import org.apache.kafka.common.requests.ProduceRequest; -import org.apache.kafka.common.requests.RequestHeader; -import org.apache.kafka.common.requests.RequestSend; -import org.apache.kafka.common.requests.ResponseHeader; -import org.apache.kafka.common.utils.MockTime; -import org.apache.kafka.test.MockSelector; -import org.apache.kafka.test.TestUtils; -import org.junit.Before; -import org.junit.Test; - -public class NetworkClientTest { - - private MockTime time = new MockTime(); - private MockSelector selector = new MockSelector(time); - private Metadata metadata = new Metadata(0, Long.MAX_VALUE); - private int nodeId = 1; - private Cluster cluster = TestUtils.singletonCluster("test", nodeId); - private Node node = cluster.nodes().get(0); - private NetworkClient client = new NetworkClient(selector, metadata, "mock", Integer.MAX_VALUE, 0, 64 * 1024, 64 * 1024); - - @Before - public void setup() { - metadata.update(cluster, time.milliseconds()); - } - - @Test - public void testReadyAndDisconnect() { - List reqs = new ArrayList(); - assertFalse("Client begins unready as it has no connection.", client.ready(node, time.milliseconds())); - assertEquals("The connection is established as a side-effect of the readiness check", 1, selector.connected().size()); - client.poll(reqs, 1, time.milliseconds()); - selector.clear(); - assertTrue("Now the client is ready", client.ready(node, time.milliseconds())); - selector.disconnect(node.id()); - client.poll(reqs, 1, time.milliseconds()); - selector.clear(); - assertFalse("After we forced the disconnection the client is no longer ready.", client.ready(node, time.milliseconds())); - assertTrue("Metadata should get updated.", metadata.needsUpdate(time.milliseconds())); - } - - @Test(expected = IllegalStateException.class) - public void testSendToUnreadyNode() { - RequestSend send = new RequestSend(5, - client.nextRequestHeader(ApiKeys.METADATA), - new MetadataRequest(Arrays.asList("test")).toStruct()); - ClientRequest request = new ClientRequest(time.milliseconds(), false, send, null); - client.poll(Arrays.asList(request), 1, time.milliseconds()); - } - - @Test - public void testSimpleRequestResponse() { - ProduceRequest produceRequest = new ProduceRequest((short) 1, 1000); - RequestHeader reqHeader = client.nextRequestHeader(ApiKeys.PRODUCE); - RequestSend send = new RequestSend(node.id(), reqHeader, produceRequest.toStruct()); - ClientRequest request = new ClientRequest(time.milliseconds(), true, send, null); - awaitReady(client, node); - client.poll(Arrays.asList(request), 1, time.milliseconds()); - assertEquals(1, client.inFlightRequestCount()); - ResponseHeader respHeader = new ResponseHeader(reqHeader.correlationId()); - Struct resp = new Struct(ProtoUtils.currentResponseSchema(ApiKeys.PRODUCE.id)); - resp.set("responses", new Object[0]); - int size = respHeader.sizeOf() + resp.sizeOf(); - ByteBuffer buffer = ByteBuffer.allocate(size); - respHeader.writeTo(buffer); - resp.writeTo(buffer); - buffer.flip(); - selector.completeReceive(new NetworkReceive(node.id(), buffer)); - List responses = client.poll(new ArrayList(), 1, time.milliseconds()); - assertEquals(1, responses.size()); - ClientResponse response = responses.get(0); - assertTrue("Should have a response body.", response.hasResponse()); - assertEquals("Should be correlated to the original request", request, response.request()); - } - - private void awaitReady(NetworkClient client, Node node) { - while (!client.ready(node, time.milliseconds())) - client.poll(new ArrayList(), 1, time.milliseconds()); - } - -} diff --git a/clients/src/test/java/org/apache/kafka/clients/producer/BufferPoolTest.java b/clients/src/test/java/org/apache/kafka/clients/producer/BufferPoolTest.java index fe3c13f..f227b5c 100644 --- a/clients/src/test/java/org/apache/kafka/clients/producer/BufferPoolTest.java +++ b/clients/src/test/java/org/apache/kafka/clients/producer/BufferPoolTest.java @@ -16,11 +16,9 @@ */ package org.apache.kafka.clients.producer; -import org.apache.kafka.clients.producer.internals.BufferPool; -import org.apache.kafka.common.metrics.Metrics; -import org.apache.kafka.common.utils.MockTime; -import org.apache.kafka.test.TestUtils; -import org.junit.Test; +import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertTrue; +import static org.junit.Assert.fail; import java.nio.ByteBuffer; import java.util.ArrayList; @@ -28,11 +26,13 @@ import java.util.List; import java.util.concurrent.CountDownLatch; import java.util.concurrent.atomic.AtomicBoolean; -import static org.junit.Assert.*; + +import org.apache.kafka.clients.producer.BufferExhaustedException; +import org.apache.kafka.clients.producer.internals.BufferPool; +import org.apache.kafka.test.TestUtils; +import org.junit.Test; public class BufferPoolTest { - private MockTime time = new MockTime(); - private Metrics metrics = new Metrics(time); /** * Test the simple non-blocking allocation paths @@ -41,7 +41,7 @@ public class BufferPoolTest { public void testSimple() throws Exception { int totalMemory = 64 * 1024; int size = 1024; - BufferPool pool = new BufferPool(totalMemory, size, false, metrics, time); + BufferPool pool = new BufferPool(totalMemory, size, false); ByteBuffer buffer = pool.allocate(size); assertEquals("Buffer size should equal requested size.", size, buffer.limit()); assertEquals("Unallocated memory should have shrunk", totalMemory - size, pool.unallocatedMemory()); @@ -68,7 +68,7 @@ public class BufferPoolTest { */ @Test(expected = IllegalArgumentException.class) public void testCantAllocateMoreMemoryThanWeHave() throws Exception { - BufferPool pool = new BufferPool(1024, 512, true, metrics, time); + BufferPool pool = new BufferPool(1024, 512, true); ByteBuffer buffer = pool.allocate(1024); assertEquals(1024, buffer.limit()); pool.deallocate(buffer); @@ -77,7 +77,7 @@ public class BufferPoolTest { @Test public void testNonblockingMode() throws Exception { - BufferPool pool = new BufferPool(2, 1, false, metrics, time); + BufferPool pool = new BufferPool(2, 1, false); pool.allocate(1); try { pool.allocate(2); @@ -92,7 +92,7 @@ public class BufferPoolTest { */ @Test public void testDelayedAllocation() throws Exception { - BufferPool pool = new BufferPool(5 * 1024, 1024, true, metrics, time); + BufferPool pool = new BufferPool(5 * 1024, 1024, true); ByteBuffer buffer = pool.allocate(1024); CountDownLatch doDealloc = asyncDeallocate(pool, buffer); CountDownLatch allocation = asyncAllocate(pool, 5 * 1024); @@ -141,7 +141,7 @@ public class BufferPoolTest { final int iterations = 50000; final int poolableSize = 1024; final int totalMemory = numThreads / 2 * poolableSize; - final BufferPool pool = new BufferPool(totalMemory, poolableSize, true, metrics, time); + final BufferPool pool = new BufferPool(totalMemory, poolableSize, true); List threads = new ArrayList(); for (int i = 0; i < numThreads; i++) threads.add(new StressTestThread(pool, iterations)); diff --git a/clients/src/test/java/org/apache/kafka/clients/producer/RecordAccumulatorTest.java b/clients/src/test/java/org/apache/kafka/clients/producer/RecordAccumulatorTest.java index 93b58d0..c4072ae 100644 --- a/clients/src/test/java/org/apache/kafka/clients/producer/RecordAccumulatorTest.java +++ b/clients/src/test/java/org/apache/kafka/clients/producer/RecordAccumulatorTest.java @@ -17,12 +17,7 @@ import static org.junit.Assert.assertEquals; import static org.junit.Assert.assertFalse; import java.nio.ByteBuffer; -import java.util.ArrayList; -import java.util.Arrays; -import java.util.Collections; -import java.util.Iterator; -import java.util.List; -import java.util.Set; +import java.util.*; import org.apache.kafka.clients.producer.internals.RecordAccumulator; import org.apache.kafka.clients.producer.internals.RecordBatch; diff --git a/clients/src/test/java/org/apache/kafka/clients/producer/SenderTest.java b/clients/src/test/java/org/apache/kafka/clients/producer/SenderTest.java index 5489aca..3ef692c 100644 --- a/clients/src/test/java/org/apache/kafka/clients/producer/SenderTest.java +++ b/clients/src/test/java/org/apache/kafka/clients/producer/SenderTest.java @@ -16,48 +16,62 @@ import static org.junit.Assert.assertEquals; import static org.junit.Assert.assertTrue; import static org.junit.Assert.fail; +import java.nio.ByteBuffer; import java.util.concurrent.ExecutionException; import java.util.concurrent.Future; -import org.apache.kafka.clients.MockClient; import org.apache.kafka.clients.producer.internals.Metadata; import org.apache.kafka.clients.producer.internals.RecordAccumulator; import org.apache.kafka.clients.producer.internals.Sender; import org.apache.kafka.common.Cluster; import org.apache.kafka.common.TopicPartition; import org.apache.kafka.common.metrics.Metrics; +import org.apache.kafka.common.network.NetworkReceive; import org.apache.kafka.common.protocol.ApiKeys; import org.apache.kafka.common.protocol.Errors; import org.apache.kafka.common.protocol.ProtoUtils; import org.apache.kafka.common.protocol.types.Struct; import org.apache.kafka.common.record.CompressionType; +import org.apache.kafka.common.requests.RequestSend; +import org.apache.kafka.common.requests.ResponseHeader; import org.apache.kafka.common.utils.MockTime; +import org.apache.kafka.test.MockSelector; import org.apache.kafka.test.TestUtils; import org.junit.Before; import org.junit.Test; public class SenderTest { + private static final String CLIENT_ID = ""; private static final int MAX_REQUEST_SIZE = 1024 * 1024; + private static final long RECONNECT_BACKOFF_MS = 0L; private static final short ACKS_ALL = -1; private static final int MAX_RETRIES = 0; private static final int REQUEST_TIMEOUT_MS = 10000; + private static final int SEND_BUFFER_SIZE = 64 * 1024; + private static final int RECEIVE_BUFFER_SIZE = 64 * 1024; + private static final int MAX_IN_FLIGHT_REQS = Integer.MAX_VALUE; private TopicPartition tp = new TopicPartition("test", 0); private MockTime time = new MockTime(); - private MockClient client = new MockClient(time); + private MockSelector selector = new MockSelector(time); private int batchSize = 16 * 1024; private Metadata metadata = new Metadata(0, Long.MAX_VALUE); private Cluster cluster = TestUtils.singletonCluster("test", 1); private Metrics metrics = new Metrics(time); private RecordAccumulator accumulator = new RecordAccumulator(batchSize, 1024 * 1024, 0L, 0L, false, metrics, time); - private Sender sender = new Sender(client, + private Sender sender = new Sender(selector, metadata, this.accumulator, + CLIENT_ID, MAX_REQUEST_SIZE, + RECONNECT_BACKOFF_MS, ACKS_ALL, MAX_RETRIES, REQUEST_TIMEOUT_MS, + SEND_BUFFER_SIZE, + RECEIVE_BUFFER_SIZE, + MAX_IN_FLIGHT_REQS, metrics, time); @@ -68,14 +82,21 @@ public class SenderTest { @Test public void testSimple() throws Exception { - int offset = 0; Future future = accumulator.append(tp, "key".getBytes(), "value".getBytes(), CompressionType.NONE, null); - sender.run(time.milliseconds()); // connect - sender.run(time.milliseconds()); // send produce request - assertEquals("We should have a single produce request in flight.", 1, client.inFlightRequestCount()); - client.respond(produceResponse(tp.topic(), tp.partition(), offset, Errors.NONE.code())); sender.run(time.milliseconds()); - assertEquals("All requests completed.", offset, client.inFlightRequestCount()); + assertEquals("We should have connected", 1, selector.connected().size()); + selector.clear(); + sender.run(time.milliseconds()); + assertEquals("Single request should be sent", 1, selector.completedSends().size()); + RequestSend request = (RequestSend) selector.completedSends().get(0); + selector.clear(); + long offset = 42; + selector.completeReceive(produceResponse(request.header().correlationId(), + cluster.leaderFor(tp).id(), + tp.topic(), + tp.partition(), + offset, + Errors.NONE.code())); sender.run(time.milliseconds()); assertTrue("Request should be completed", future.isDone()); assertEquals(offset, future.get().offset()); @@ -85,43 +106,69 @@ public class SenderTest { public void testRetries() throws Exception { // create a sender with retries = 1 int maxRetries = 1; - Sender sender = new Sender(client, + Sender sender = new Sender(selector, metadata, this.accumulator, + CLIENT_ID, MAX_REQUEST_SIZE, + RECONNECT_BACKOFF_MS, ACKS_ALL, maxRetries, REQUEST_TIMEOUT_MS, + SEND_BUFFER_SIZE, + RECEIVE_BUFFER_SIZE, + MAX_IN_FLIGHT_REQS, new Metrics(), time); - // do a successful retry Future future = accumulator.append(tp, "key".getBytes(), "value".getBytes(), CompressionType.NONE, null); - sender.run(time.milliseconds()); // connect - sender.run(time.milliseconds()); // send produce request - assertEquals(1, client.inFlightRequestCount()); - client.disconnect(client.requests().peek().request().destination()); - assertEquals(0, client.inFlightRequestCount()); - sender.run(time.milliseconds()); // receive error - sender.run(time.milliseconds()); // reconnect - sender.run(time.milliseconds()); // resend - assertEquals(1, client.inFlightRequestCount()); - int offset = 0; - client.respond(produceResponse(tp.topic(), tp.partition(), offset, Errors.NONE.code())); + RequestSend request1 = completeSend(sender); + selector.clear(); + selector.completeReceive(produceResponse(request1.header().correlationId(), + cluster.leaderFor(tp).id(), + tp.topic(), + tp.partition(), + -1, + Errors.REQUEST_TIMED_OUT.code())); sender.run(time.milliseconds()); - assertTrue("Request should have retried and completed", future.isDone()); - assertEquals(offset, future.get().offset()); + selector.clear(); + sender.run(time.milliseconds()); + RequestSend request2 = completeSend(sender); + selector.completeReceive(produceResponse(request2.header().correlationId(), + cluster.leaderFor(tp).id(), + tp.topic(), + tp.partition(), + 42, + Errors.NONE.code())); + sender.run(time.milliseconds()); + assertTrue("Request should retry and complete", future.isDone()); + assertEquals(42, future.get().offset()); + } - // do an unsuccessful retry - future = accumulator.append(tp, "key".getBytes(), "value".getBytes(), CompressionType.NONE, null); - sender.run(time.milliseconds()); // send produce request - for (int i = 0; i < maxRetries + 1; i++) { - client.disconnect(client.requests().peek().request().destination()); - sender.run(time.milliseconds()); // receive error - sender.run(time.milliseconds()); // reconnect - sender.run(time.milliseconds()); // resend - } + @Test + public void testMetadataRefreshOnNoLeaderException() throws Exception { + Future future = accumulator.append(tp, "key".getBytes(), "value".getBytes(), CompressionType.NONE, null); + RequestSend request = completeSend(); + selector.clear(); + selector.completeReceive(produceResponse(request.header().correlationId(), + cluster.leaderFor(tp).id(), + tp.topic(), + tp.partition(), + -1, + Errors.NOT_LEADER_FOR_PARTITION.code())); + sender.run(time.milliseconds()); + completedWithError(future, Errors.NOT_LEADER_FOR_PARTITION); + assertTrue("Error triggers a metadata update.", metadata.needsUpdate(time.milliseconds())); + } + + @Test + public void testMetadataRefreshOnDisconnect() throws Exception { + Future future = accumulator.append(tp, "key".getBytes(), "value".getBytes(), CompressionType.NONE, null); + completeSend(); + selector.clear(); + selector.disconnect(cluster.leaderFor(tp).id()); sender.run(time.milliseconds()); completedWithError(future, Errors.NETWORK_EXCEPTION); + assertTrue("The disconnection triggers a metadata update.", metadata.needsUpdate(time.milliseconds())); } private void completedWithError(Future future, Errors error) throws Exception { @@ -134,7 +181,17 @@ public class SenderTest { } } - private Struct produceResponse(String topic, int part, long offset, int error) { + private RequestSend completeSend() { + return completeSend(sender); + } + + private RequestSend completeSend(Sender sender) { + while (selector.completedSends().size() == 0) + sender.run(time.milliseconds()); + return (RequestSend) selector.completedSends().get(0); + } + + private NetworkReceive produceResponse(int correlation, int source, String topic, int part, long offset, int error) { Struct struct = new Struct(ProtoUtils.currentResponseSchema(ApiKeys.PRODUCE.id)); Struct response = struct.instance("responses"); response.set("topic", topic); @@ -144,7 +201,12 @@ public class SenderTest { partResp.set("base_offset", offset); response.set("partition_responses", new Object[] { partResp }); struct.set("responses", new Object[] { response }); - return struct; + ResponseHeader header = new ResponseHeader(correlation); + ByteBuffer buffer = ByteBuffer.allocate(header.sizeOf() + struct.sizeOf()); + header.writeTo(buffer); + struct.writeTo(buffer); + buffer.rewind(); + return new NetworkReceive(source, buffer); } } diff --git a/clients/src/test/java/org/apache/kafka/common/metrics/MetricsTest.java b/clients/src/test/java/org/apache/kafka/common/metrics/MetricsTest.java index 19bea0f..e4e0a04 100644 --- a/clients/src/test/java/org/apache/kafka/common/metrics/MetricsTest.java +++ b/clients/src/test/java/org/apache/kafka/common/metrics/MetricsTest.java @@ -213,7 +213,7 @@ public class MetricsTest { public double value = 0.0; @Override - public double measure(MetricConfig config, long now) { + public double measure(MetricConfig config, long nowMs) { return value; } diff --git a/clients/src/test/java/org/apache/kafka/common/utils/MockTime.java b/clients/src/test/java/org/apache/kafka/common/utils/MockTime.java index eb7fcf0..cda8e64 100644 --- a/clients/src/test/java/org/apache/kafka/common/utils/MockTime.java +++ b/clients/src/test/java/org/apache/kafka/common/utils/MockTime.java @@ -1,22 +1,25 @@ /** - * 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. + * 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.utils; import java.util.concurrent.TimeUnit; -/** - * A clock that you can manually advance by calling sleep - */ +import org.apache.kafka.common.utils.Time; + public class MockTime implements Time { private long nanos = 0; diff --git a/core/src/main/scala/kafka/cluster/Partition.scala b/core/src/main/scala/kafka/cluster/Partition.scala index a9c0465..5c04f3e 100644 --- a/core/src/main/scala/kafka/cluster/Partition.scala +++ b/core/src/main/scala/kafka/cluster/Partition.scala @@ -18,7 +18,7 @@ package kafka.cluster import kafka.common._ import kafka.admin.AdminUtils -import kafka.utils.{ZkUtils, ReplicationUtils, Pool, Time, Logging} +import kafka.utils.{ReplicationUtils, Pool, Time, Logging} import kafka.utils.Utils.inLock import kafka.api.{PartitionStateInfo, LeaderAndIsr} import kafka.log.LogConfig @@ -261,8 +261,9 @@ class Partition(val topic: String, info("Expanding ISR for partition [%s,%d] from %s to %s" .format(topic, partitionId, inSyncReplicas.map(_.brokerId).mkString(","), newInSyncReplicas.map(_.brokerId).mkString(","))) // update ISR in ZK and cache - val (updateSucceeded,newVersion) = ReplicationUtils.updateIsr(zkClient, topic, partitionId, localBrokerId, - leaderEpoch, controllerEpoch, zkVersion, newInSyncReplicas) + val newLeaderAndIsr = new LeaderAndIsr(localBrokerId, leaderEpoch, newInSyncReplicas.map(r => r.brokerId).toList, zkVersion) + val (updateSucceeded,newVersion) = ReplicationUtils.updateLeaderAndIsr(zkClient, topic, partitionId, + newLeaderAndIsr, controllerEpoch, zkVersion) if(updateSucceeded) { inSyncReplicas = newInSyncReplicas zkVersion = newVersion @@ -333,8 +334,9 @@ class Partition(val topic: String, info("Shrinking ISR for partition [%s,%d] from %s to %s".format(topic, partitionId, inSyncReplicas.map(_.brokerId).mkString(","), newInSyncReplicas.map(_.brokerId).mkString(","))) // update ISR in zk and in cache - val (updateSucceeded,newVersion) = ReplicationUtils.updateIsr(zkClient, topic, partitionId, localBrokerId, - leaderEpoch, controllerEpoch, zkVersion, newInSyncReplicas) + val newLeaderAndIsr = new LeaderAndIsr(localBrokerId, leaderEpoch, newInSyncReplicas.map(r => r.brokerId).toList, zkVersion) + val (updateSucceeded,newVersion) = ReplicationUtils.updateLeaderAndIsr(zkClient, topic, partitionId, + newLeaderAndIsr, controllerEpoch, zkVersion) if(updateSucceeded) { inSyncReplicas = newInSyncReplicas zkVersion = newVersion diff --git a/core/src/main/scala/kafka/controller/KafkaController.scala b/core/src/main/scala/kafka/controller/KafkaController.scala index 8af48ab..2012f05 100644 --- a/core/src/main/scala/kafka/controller/KafkaController.scala +++ b/core/src/main/scala/kafka/controller/KafkaController.scala @@ -249,24 +249,22 @@ class KafkaController(val config : KafkaConfig, zkClient: ZkClient, val brokerSt // Move leadership serially to relinquish lock. inLock(controllerContext.controllerLock) { controllerContext.partitionLeadershipInfo.get(topicAndPartition).foreach { currLeaderIsrAndControllerEpoch => - if (replicationFactor > 1) { - if (currLeaderIsrAndControllerEpoch.leaderAndIsr.leader == id) { - // If the broker leads the topic partition, transition the leader and update isr. Updates zk and - // notifies all affected brokers - partitionStateMachine.handleStateChanges(Set(topicAndPartition), OnlinePartition, - controlledShutdownPartitionLeaderSelector) - } else { - // Stop the replica first. The state change below initiates ZK changes which should take some time - // before which the stop replica request should be completed (in most cases) - brokerRequestBatch.newBatch() - brokerRequestBatch.addStopReplicaRequestForBrokers(Seq(id), topicAndPartition.topic, - topicAndPartition.partition, deletePartition = false) - brokerRequestBatch.sendRequestsToBrokers(epoch, controllerContext.correlationId.getAndIncrement) - - // If the broker is a follower, updates the isr in ZK and notifies the current leader - replicaStateMachine.handleStateChanges(Set(PartitionAndReplica(topicAndPartition.topic, - topicAndPartition.partition, id)), OfflineReplica) - } + if (currLeaderIsrAndControllerEpoch.leaderAndIsr.leader == id && replicationFactor > 1) { + // If the broker leads the topic partition, transition the leader and update isr. Updates zk and + // notifies all affected brokers + partitionStateMachine.handleStateChanges(Set(topicAndPartition), OnlinePartition, + controlledShutdownPartitionLeaderSelector) + } else { + // Stop the replica first. The state change below initiates ZK changes which should take some time + // before which the stop replica request should be completed (in most cases) + brokerRequestBatch.newBatch() + brokerRequestBatch.addStopReplicaRequestForBrokers(Seq(id), topicAndPartition.topic, + topicAndPartition.partition, deletePartition = false) + brokerRequestBatch.sendRequestsToBrokers(epoch, controllerContext.correlationId.getAndIncrement) + + // If the broker is a follower, updates the isr in ZK and notifies the current leader + replicaStateMachine.handleStateChanges(Set(PartitionAndReplica(topicAndPartition.topic, + topicAndPartition.partition, id)), OfflineReplica) } } } @@ -953,7 +951,7 @@ class KafkaController(val config : KafkaConfig, zkClient: ZkClient, val brokerSt var zkWriteCompleteOrUnnecessary = false while (!zkWriteCompleteOrUnnecessary) { // refresh leader and isr from zookeeper again - val leaderIsrAndEpochOpt = ZkUtils.getLeaderIsrAndEpochForPartition(zkClient, topic, partition) + val leaderIsrAndEpochOpt = ReplicationUtils.getLeaderIsrAndEpochForPartition(zkClient, topic, partition) zkWriteCompleteOrUnnecessary = leaderIsrAndEpochOpt match { case Some(leaderIsrAndEpoch) => // increment the leader epoch even if the ISR changes val leaderAndIsr = leaderIsrAndEpoch.leaderAndIsr @@ -979,13 +977,10 @@ class KafkaController(val config : KafkaConfig, zkClient: ZkClient, val brokerSt val newLeaderAndIsr = new LeaderAndIsr(newLeader, leaderAndIsr.leaderEpoch + 1, newIsr, leaderAndIsr.zkVersion + 1) // update the new leadership decision in zookeeper or retry - val (updateSucceeded, newVersion) = ZkUtils.conditionalUpdatePersistentPath( - zkClient, - ZkUtils.getTopicPartitionLeaderAndIsrPath(topic, partition), - ZkUtils.leaderAndIsrZkData(newLeaderAndIsr, epoch), - leaderAndIsr.zkVersion,Some(ReplicationUtils.checkLeaderAndIsrZkData)) - newLeaderAndIsr.zkVersion = newVersion + val (updateSucceeded, newVersion) = ReplicationUtils.updateLeaderAndIsr(zkClient, topic, partition, + newLeaderAndIsr, epoch, leaderAndIsr.zkVersion) + newLeaderAndIsr.zkVersion = newVersion finalLeaderIsrAndControllerEpoch = Some(LeaderIsrAndControllerEpoch(newLeaderAndIsr, epoch)) controllerContext.partitionLeadershipInfo.put(topicAndPartition, finalLeaderIsrAndControllerEpoch.get) if (updateSucceeded) @@ -1019,7 +1014,7 @@ class KafkaController(val config : KafkaConfig, zkClient: ZkClient, val brokerSt var zkWriteCompleteOrUnnecessary = false while (!zkWriteCompleteOrUnnecessary) { // refresh leader and isr from zookeeper again - val leaderIsrAndEpochOpt = ZkUtils.getLeaderIsrAndEpochForPartition(zkClient, topic, partition) + val leaderIsrAndEpochOpt = ReplicationUtils.getLeaderIsrAndEpochForPartition(zkClient, topic, partition) zkWriteCompleteOrUnnecessary = leaderIsrAndEpochOpt match { case Some(leaderIsrAndEpoch) => val leaderAndIsr = leaderIsrAndEpoch.leaderAndIsr @@ -1033,11 +1028,9 @@ class KafkaController(val config : KafkaConfig, zkClient: ZkClient, val brokerSt val newLeaderAndIsr = new LeaderAndIsr(leaderAndIsr.leader, leaderAndIsr.leaderEpoch + 1, leaderAndIsr.isr, leaderAndIsr.zkVersion + 1) // update the new leadership decision in zookeeper or retry - val (updateSucceeded, newVersion) = ZkUtils.conditionalUpdatePersistentPath( - zkClient, - ZkUtils.getTopicPartitionLeaderAndIsrPath(topic, partition), - ZkUtils.leaderAndIsrZkData(newLeaderAndIsr, epoch), - leaderAndIsr.zkVersion,Some(ReplicationUtils.checkLeaderAndIsrZkData)) + val (updateSucceeded, newVersion) = ReplicationUtils.updateLeaderAndIsr(zkClient, topic, + partition, newLeaderAndIsr, epoch, leaderAndIsr.zkVersion) + newLeaderAndIsr.zkVersion = newVersion finalLeaderIsrAndControllerEpoch = Some(LeaderIsrAndControllerEpoch(newLeaderAndIsr, epoch)) if (updateSucceeded) @@ -1335,16 +1328,6 @@ case class LeaderIsrAndControllerEpoch(val leaderAndIsr: LeaderAndIsr, controlle leaderAndIsrInfo.append(",ControllerEpoch:" + controllerEpoch + ")") leaderAndIsrInfo.toString() } - - override def equals(obj: Any): Boolean = { - obj match { - case null => false - case n: LeaderIsrAndControllerEpoch => - leaderAndIsr.leader == n.leaderAndIsr.leader && leaderAndIsr.isr.sorted == n.leaderAndIsr.isr.sorted && - leaderAndIsr.leaderEpoch == n.leaderAndIsr.leaderEpoch && controllerEpoch == n.controllerEpoch - case _ => false - } - } } object ControllerStats extends KafkaMetricsGroup { diff --git a/core/src/main/scala/kafka/controller/PartitionStateMachine.scala b/core/src/main/scala/kafka/controller/PartitionStateMachine.scala index e29e470..8c816e8 100644 --- a/core/src/main/scala/kafka/controller/PartitionStateMachine.scala +++ b/core/src/main/scala/kafka/controller/PartitionStateMachine.scala @@ -293,7 +293,7 @@ class PartitionStateMachine(controller: KafkaController) extends Logging { } catch { case e: ZkNodeExistsException => // read the controller epoch - val leaderIsrAndEpoch = ZkUtils.getLeaderIsrAndEpochForPartition(zkClient, topicAndPartition.topic, + val leaderIsrAndEpoch = ReplicationUtils.getLeaderIsrAndEpochForPartition(zkClient, topicAndPartition.topic, topicAndPartition.partition).get val failMsg = ("encountered error while changing partition %s's state from New to Online since LeaderAndIsr path already " + "exists with value %s and controller epoch %d") @@ -383,7 +383,7 @@ class PartitionStateMachine(controller: KafkaController) extends Logging { private def getLeaderIsrAndEpochOrThrowException(topic: String, partition: Int): LeaderIsrAndControllerEpoch = { val topicAndPartition = TopicAndPartition(topic, partition) - ZkUtils.getLeaderIsrAndEpochForPartition(zkClient, topic, partition) match { + ReplicationUtils.getLeaderIsrAndEpochForPartition(zkClient, topic, partition) match { case Some(currentLeaderIsrAndEpoch) => currentLeaderIsrAndEpoch case None => val failMsg = "LeaderAndIsr information doesn't exist for partition %s in %s state" diff --git a/core/src/main/scala/kafka/controller/ReplicaStateMachine.scala b/core/src/main/scala/kafka/controller/ReplicaStateMachine.scala index 2f0f29d..ad9c7c4 100644 --- a/core/src/main/scala/kafka/controller/ReplicaStateMachine.scala +++ b/core/src/main/scala/kafka/controller/ReplicaStateMachine.scala @@ -20,7 +20,7 @@ import collection._ import collection.JavaConversions._ import java.util.concurrent.atomic.AtomicBoolean import kafka.common.{TopicAndPartition, StateChangeFailedException} -import kafka.utils.{ZkUtils, Logging} +import kafka.utils.{ZkUtils, ReplicationUtils, Logging} import org.I0Itec.zkclient.IZkChildListener import org.apache.log4j.Logger import kafka.controller.Callbacks._ @@ -153,7 +153,7 @@ class ReplicaStateMachine(controller: KafkaController) extends Logging { case NewReplica => assertValidPreviousStates(partitionAndReplica, List(NonExistentReplica), targetState) // start replica as a follower to the current leader for its partition - val leaderIsrAndControllerEpochOpt = ZkUtils.getLeaderIsrAndEpochForPartition(zkClient, topic, partition) + val leaderIsrAndControllerEpochOpt = ReplicationUtils.getLeaderIsrAndEpochForPartition(zkClient, topic, partition) leaderIsrAndControllerEpochOpt match { case Some(leaderIsrAndControllerEpoch) => if(leaderIsrAndControllerEpoch.leaderAndIsr.leader == replicaId) @@ -367,5 +367,3 @@ case object ReplicaDeletionStarted extends ReplicaState { val state: Byte = 4} case object ReplicaDeletionSuccessful extends ReplicaState { val state: Byte = 5} case object ReplicaDeletionIneligible extends ReplicaState { val state: Byte = 6} case object NonExistentReplica extends ReplicaState { val state: Byte = 7 } - - diff --git a/core/src/main/scala/kafka/utils/ReplicationUtils.scala b/core/src/main/scala/kafka/utils/ReplicationUtils.scala index eb53837..8b8dbb5 100644 --- a/core/src/main/scala/kafka/utils/ReplicationUtils.scala +++ b/core/src/main/scala/kafka/utils/ReplicationUtils.scala @@ -16,7 +16,7 @@ */ package kafka.utils -import kafka.cluster.Replica + import kafka.api.LeaderAndIsr import kafka.controller.LeaderIsrAndControllerEpoch import org.apache.zookeeper.data.Stat @@ -27,31 +27,27 @@ import scala.collection._ object ReplicationUtils extends Logging { - def updateIsr(zkClient: ZkClient, topic: String, partitionId: Int, brokerId: Int, leaderEpoch: Int, - controllerEpoch: Int, zkVersion: Int, newIsr: Set[Replica]): (Boolean,Int) = { - debug("Updated ISR for partition [%s,%d] to %s".format(topic, partitionId, newIsr.mkString(","))) - val newLeaderAndIsr = new LeaderAndIsr(brokerId, leaderEpoch, newIsr.map(r => r.brokerId).toList, zkVersion) + def updateLeaderAndIsr(zkClient: ZkClient, topic: String, partitionId: Int, newLeaderAndIsr: LeaderAndIsr, controllerEpoch: Int, + zkVersion: Int): (Boolean,Int) = { + debug("Updated ISR for partition [%s,%d] to %s".format(topic, partitionId, newLeaderAndIsr.isr.mkString(","))) val path = ZkUtils.getTopicPartitionLeaderAndIsrPath(topic, partitionId) val newLeaderData = ZkUtils.leaderAndIsrZkData(newLeaderAndIsr, controllerEpoch) // use the epoch of the controller that made the leadership decision, instead of the current controller epoch - ZkUtils.conditionalUpdatePersistentPath(zkClient, path, newLeaderData, zkVersion, - Some(checkLeaderAndIsrZkData)) + ZkUtils.conditionalUpdatePersistentPath(zkClient, path, newLeaderData, zkVersion, Some(checkLeaderAndIsrZkData)) } - def checkLeaderAndIsrZkData(zkClient: ZkClient, path: String,newLeaderData: String, zkVersion: Int): (Boolean,Int) = { + def checkLeaderAndIsrZkData(zkClient: ZkClient, path: String, expectedLeaderAndIsrInfo: String): (Boolean,Int) = { try { - val newLeaderStat: Stat = new Stat() - newLeaderStat.setVersion(zkVersion) - val newLeader = parseLeaderAndIsr(newLeaderData, path, newLeaderStat) - val writtenLeaderAndIsrInfo = ZkUtils.readDataMaybeNull(zkClient,path) + val writtenLeaderAndIsrInfo = ZkUtils.readDataMaybeNull(zkClient, path) val writtenLeaderOpt = writtenLeaderAndIsrInfo._1 val writtenStat = writtenLeaderAndIsrInfo._2 + val newLeader = parseLeaderAndIsr(expectedLeaderAndIsrInfo, path, writtenStat) writtenLeaderOpt match { case Some(writtenData) => val writtenLeader = parseLeaderAndIsr(writtenData, path, writtenStat) - (newLeader,writtenLeader) match { + (newLeader,writtenLeader) match { case (Some(newLeader),Some(writtenLeader)) => - if(newLeader.equals(writtenLeader)) + if(newLeader == writtenLeader) return (true,writtenStat.getVersion()) case _ => } @@ -63,7 +59,18 @@ object ReplicationUtils extends Logging { (false,-1) } - def parseLeaderAndIsr(leaderAndIsrStr: String, path: String, stat: Stat) + def getLeaderIsrAndEpochForPartition(zkClient: ZkClient, topic: String, partition: Int):Option[LeaderIsrAndControllerEpoch] = { + val leaderAndIsrPath = ZkUtils.getTopicPartitionLeaderAndIsrPath(topic, partition) + val leaderAndIsrInfo = ZkUtils.readDataMaybeNull(zkClient, leaderAndIsrPath) + val leaderAndIsrOpt = leaderAndIsrInfo._1 + val stat = leaderAndIsrInfo._2 + leaderAndIsrOpt match { + case Some(leaderAndIsrStr) => parseLeaderAndIsr(leaderAndIsrStr, leaderAndIsrPath, stat) + case None => None + } + } + + private def parseLeaderAndIsr(leaderAndIsrStr: String, path: String, stat: Stat) : Option[LeaderIsrAndControllerEpoch] = { Json.parseFull(leaderAndIsrStr) match { case Some(m) => diff --git a/core/src/main/scala/kafka/utils/ZkUtils.scala b/core/src/main/scala/kafka/utils/ZkUtils.scala index 1a23eb4..be3bf12 100644 --- a/core/src/main/scala/kafka/utils/ZkUtils.scala +++ b/core/src/main/scala/kafka/utils/ZkUtils.scala @@ -19,24 +19,20 @@ package kafka.utils import kafka.cluster.{Broker, Cluster} import kafka.consumer.TopicCount -import org.I0Itec.zkclient.{IZkDataListener, ZkClient} +import org.I0Itec.zkclient.ZkClient import org.I0Itec.zkclient.exception.{ZkNodeExistsException, ZkNoNodeException, ZkMarshallingError, ZkBadVersionException} import org.I0Itec.zkclient.serialize.ZkSerializer import collection._ import kafka.api.LeaderAndIsr -import mutable.ListBuffer import org.apache.zookeeper.data.Stat -import java.util.concurrent.locks.{ReentrantLock, Condition} import kafka.admin._ import kafka.common.{KafkaException, NoEpochForPartitionException} import kafka.controller.ReassignedPartitionsContext -import kafka.controller.PartitionAndReplica import kafka.controller.KafkaController -import scala.{collection, Some} +import scala.Some import kafka.controller.LeaderIsrAndControllerEpoch import kafka.common.TopicAndPartition -import kafka.utils.Utils.inLock import scala.collection object ZkUtils extends Logging { @@ -86,19 +82,8 @@ object ZkUtils extends Logging { brokerIds.map(_.toInt).map(getBrokerInfo(zkClient, _)).filter(_.isDefined).map(_.get) } - def getLeaderIsrAndEpochForPartition(zkClient: ZkClient, topic: String, partition: Int):Option[LeaderIsrAndControllerEpoch] = { - val leaderAndIsrPath = getTopicPartitionLeaderAndIsrPath(topic, partition) - val leaderAndIsrInfo = readDataMaybeNull(zkClient, leaderAndIsrPath) - val leaderAndIsrOpt = leaderAndIsrInfo._1 - val stat = leaderAndIsrInfo._2 - leaderAndIsrOpt match { - case Some(leaderAndIsrStr) => ReplicationUtils.parseLeaderAndIsr(leaderAndIsrStr, leaderAndIsrPath, stat) - case None => None - } - } - def getLeaderAndIsrForPartition(zkClient: ZkClient, topic: String, partition: Int):Option[LeaderAndIsr] = { - getLeaderIsrAndEpochForPartition(zkClient, topic, partition).map(_.leaderAndIsr) + ReplicationUtils.getLeaderIsrAndEpochForPartition(zkClient, topic, partition).map(_.leaderAndIsr) } def setupCommonPaths(zkClient: ZkClient) { @@ -363,26 +348,29 @@ object ZkUtils extends Logging { /** * Conditional update the persistent path data, return (true, newVersion) if it succeeds, otherwise (the path doesn't * exist, the current version is not the expected version, etc.) return (false, -1) + * + * When there is a ConnectionLossException during the conditional update, zkClient will retry the update and may fail + * since the previous update may have succeeded (but the stored zkVersion no longer matches the expected one). + * In this case, we will run the optionalChecker to further check if the write indeed have succeeded. */ def conditionalUpdatePersistentPath(client: ZkClient, path: String, data: String, expectVersion: Int, - optionalChecker:Option[(ZkClient, String,String,Int) => (Boolean,Int)] = None): (Boolean, Int) = { + optionalChecker:Option[(ZkClient, String, String) => (Boolean,Int)] = None): (Boolean, Int) = { try { val stat = client.writeDataReturnStat(path, data, expectVersion) debug("Conditional update of path %s with value %s and expected version %d succeeded, returning the new version: %d" .format(path, data, expectVersion, stat.getVersion)) (true, stat.getVersion) } catch { - case e1: ZkBadVersionException => { + case e1: ZkBadVersionException => optionalChecker match { - case Some(checker) => return checker(client,path,data,expectVersion) + case Some(checker) => return checker(client, path, data) case _ => debug("Checker method is not passed skipping zkData match") } - error("Conditional update of path %s with data %s and expected version %d failed due to %s".format(path, data, + warn("Conditional update of path %s with data %s and expected version %d failed due to %s".format(path, data, expectVersion, e1.getMessage)) (false, -1) - } case e2: Exception => - error("Conditional update of path %s with data %s and expected version %d failed due to %s".format(path, data, + warn("Conditional update of path %s with data %s and expected version %d failed due to %s".format(path, data, expectVersion, e2.getMessage)) (false, -1) } @@ -512,7 +500,7 @@ object ZkUtils extends Logging { : mutable.Map[TopicAndPartition, LeaderIsrAndControllerEpoch] = { val ret = new mutable.HashMap[TopicAndPartition, LeaderIsrAndControllerEpoch] for(topicAndPartition <- topicAndPartitions) { - ZkUtils.getLeaderIsrAndEpochForPartition(zkClient, topicAndPartition.topic, topicAndPartition.partition) match { + ReplicationUtils.getLeaderIsrAndEpochForPartition(zkClient, topicAndPartition.topic, topicAndPartition.partition) match { case Some(leaderIsrAndControllerEpoch) => ret.put(topicAndPartition, leaderIsrAndControllerEpoch) case None => } diff --git a/core/src/test/scala/integration/kafka/api/ProducerFailureHandlingTest.scala b/core/src/test/scala/integration/kafka/api/ProducerFailureHandlingTest.scala index d146444..cd4ca2f 100644 --- a/core/src/test/scala/integration/kafka/api/ProducerFailureHandlingTest.scala +++ b/core/src/test/scala/integration/kafka/api/ProducerFailureHandlingTest.scala @@ -50,8 +50,8 @@ class ProducerFailureHandlingTest extends JUnit3Suite with ZooKeeperTestHarness private var producer3: KafkaProducer = null private var producer4: KafkaProducer = null - private val props1 = TestUtils.createBrokerConfig(brokerId1, port1, false) - private val props2 = TestUtils.createBrokerConfig(brokerId2, port2, false) + private val props1 = TestUtils.createBrokerConfig(brokerId1, port1) + private val props2 = TestUtils.createBrokerConfig(brokerId2, port2) props1.put("auto.create.topics.enable", "false") props2.put("auto.create.topics.enable", "false") private val config1 = new KafkaConfig(props1) @@ -127,7 +127,7 @@ class ProducerFailureHandlingTest extends JUnit3Suite with ZooKeeperTestHarness * With non-exist-topic the future metadata should return ExecutionException caused by TimeoutException */ @Test - def testNonExistentTopic() { + def testNonExistTopic() { // send a record with non-exist topic val record = new ProducerRecord(topic2, null, "key".getBytes, "value".getBytes) intercept[ExecutionException] { @@ -333,4 +333,4 @@ class ProducerFailureHandlingTest extends JUnit3Suite with ZooKeeperTestHarness producer.close } } -} +} \ No newline at end of file diff --git a/core/src/test/scala/integration/kafka/api/ProducerSendTest.scala b/core/src/test/scala/integration/kafka/api/ProducerSendTest.scala index 34a7db4..3c2bf36 100644 --- a/core/src/test/scala/integration/kafka/api/ProducerSendTest.scala +++ b/core/src/test/scala/integration/kafka/api/ProducerSendTest.scala @@ -45,8 +45,8 @@ class ProducerSendTest extends JUnit3Suite with ZooKeeperTestHarness { private var consumer1: SimpleConsumer = null private var consumer2: SimpleConsumer = null - private val props1 = TestUtils.createBrokerConfig(brokerId1, port1, false) - private val props2 = TestUtils.createBrokerConfig(brokerId2, port2, false) + private val props1 = TestUtils.createBrokerConfig(brokerId1, port1) + private val props2 = TestUtils.createBrokerConfig(brokerId2, port2) props1.put("num.partitions", "4") props2.put("num.partitions", "4") private val config1 = new KafkaConfig(props1) @@ -255,4 +255,4 @@ class ProducerSendTest extends JUnit3Suite with ZooKeeperTestHarness { } } } -} +} \ No newline at end of file diff --git a/core/src/test/scala/unit/kafka/admin/AddPartitionsTest.scala b/core/src/test/scala/unit/kafka/admin/AddPartitionsTest.scala index 1bf2667..fcd5eee 100644 --- a/core/src/test/scala/unit/kafka/admin/AddPartitionsTest.scala +++ b/core/src/test/scala/unit/kafka/admin/AddPartitionsTest.scala @@ -37,10 +37,10 @@ class AddPartitionsTest extends JUnit3Suite with ZooKeeperTestHarness { val port3 = TestUtils.choosePort() val port4 = TestUtils.choosePort() - val configProps1 = TestUtils.createBrokerConfig(brokerId1, port1, false) - val configProps2 = TestUtils.createBrokerConfig(brokerId2, port2, false) - val configProps3 = TestUtils.createBrokerConfig(brokerId3, port3, false) - val configProps4 = TestUtils.createBrokerConfig(brokerId4, port4, false) + val configProps1 = TestUtils.createBrokerConfig(brokerId1, port1) + val configProps2 = TestUtils.createBrokerConfig(brokerId2, port2) + val configProps3 = TestUtils.createBrokerConfig(brokerId3, port3) + val configProps4 = TestUtils.createBrokerConfig(brokerId4, port4) var servers: Seq[KafkaServer] = Seq.empty[KafkaServer] var brokers: Seq[Broker] = Seq.empty[Broker] @@ -205,4 +205,4 @@ class AddPartitionsTest extends JUnit3Suite with ZooKeeperTestHarness { assertEquals(partition6DataForTopic3.replicas(2).id, 2) assertEquals(partition6DataForTopic3.replicas(3).id, 3) } -} +} \ No newline at end of file diff --git a/core/src/test/scala/unit/kafka/admin/AdminTest.scala b/core/src/test/scala/unit/kafka/admin/AdminTest.scala index e289798..4f6ddca 100644 --- a/core/src/test/scala/unit/kafka/admin/AdminTest.scala +++ b/core/src/test/scala/unit/kafka/admin/AdminTest.scala @@ -145,7 +145,7 @@ class AdminTest extends JUnit3Suite with ZooKeeperTestHarness with Logging { val expectedReplicaAssignment = Map(0 -> List(0, 1, 2)) val topic = "test" // create brokers - val servers = TestUtils.createBrokerConfigs(4, false).map(b => TestUtils.createServer(new KafkaConfig(b))) + val servers = TestUtils.createBrokerConfigs(4).map(b => TestUtils.createServer(new KafkaConfig(b))) // create the topic AdminUtils.createOrUpdateTopicPartitionAssignmentPathInZK(zkClient, topic, expectedReplicaAssignment) // reassign partition 0 @@ -176,7 +176,7 @@ class AdminTest extends JUnit3Suite with ZooKeeperTestHarness with Logging { val expectedReplicaAssignment = Map(0 -> List(0, 1, 2)) val topic = "test" // create brokers - val servers = TestUtils.createBrokerConfigs(4, false).map(b => TestUtils.createServer(new KafkaConfig(b))) + val servers = TestUtils.createBrokerConfigs(4).map(b => TestUtils.createServer(new KafkaConfig(b))) // create the topic AdminUtils.createOrUpdateTopicPartitionAssignmentPathInZK(zkClient, topic, expectedReplicaAssignment) // reassign partition 0 @@ -207,7 +207,7 @@ class AdminTest extends JUnit3Suite with ZooKeeperTestHarness with Logging { val expectedReplicaAssignment = Map(0 -> List(0, 1)) val topic = "test" // create brokers - val servers = TestUtils.createBrokerConfigs(4, false).map(b => TestUtils.createServer(new KafkaConfig(b))) + val servers = TestUtils.createBrokerConfigs(4).map(b => TestUtils.createServer(new KafkaConfig(b))) // create the topic AdminUtils.createOrUpdateTopicPartitionAssignmentPathInZK(zkClient, topic, expectedReplicaAssignment) // reassign partition 0 @@ -236,7 +236,7 @@ class AdminTest extends JUnit3Suite with ZooKeeperTestHarness with Logging { def testReassigningNonExistingPartition() { val topic = "test" // create brokers - val servers = TestUtils.createBrokerConfigs(4, false).map(b => TestUtils.createServer(new KafkaConfig(b))) + val servers = TestUtils.createBrokerConfigs(4).map(b => TestUtils.createServer(new KafkaConfig(b))) // reassign partition 0 val newReplicas = Seq(2, 3) val partitionToBeReassigned = 0 @@ -262,7 +262,7 @@ class AdminTest extends JUnit3Suite with ZooKeeperTestHarness with Logging { val reassignPartitionsCommand = new ReassignPartitionsCommand(zkClient, Map(topicAndPartition -> newReplicas)) reassignPartitionsCommand.reassignPartitions // create brokers - val servers = TestUtils.createBrokerConfigs(2, false).map(b => TestUtils.createServer(new KafkaConfig(b))) + val servers = TestUtils.createBrokerConfigs(2).map(b => TestUtils.createServer(new KafkaConfig(b))) // wait until reassignment completes TestUtils.waitUntilTrue(() => !checkIfReassignPartitionPathExists(zkClient), @@ -298,7 +298,7 @@ class AdminTest extends JUnit3Suite with ZooKeeperTestHarness with Logging { val partition = 1 val preferredReplica = 0 // create brokers - val serverConfigs = TestUtils.createBrokerConfigs(3, false).map(new KafkaConfig(_)) + val serverConfigs = TestUtils.createBrokerConfigs(3).map(new KafkaConfig(_)) // create the topic AdminUtils.createOrUpdateTopicPartitionAssignmentPathInZK(zkClient, topic, expectedReplicaAssignment) val servers = serverConfigs.reverse.map(s => TestUtils.createServer(s)) @@ -318,7 +318,7 @@ class AdminTest extends JUnit3Suite with ZooKeeperTestHarness with Logging { val topic = "test" val partition = 1 // create brokers - val serverConfigs = TestUtils.createBrokerConfigs(3, false).map(new KafkaConfig(_)) + val serverConfigs = TestUtils.createBrokerConfigs(3).map(new KafkaConfig(_)) val servers = serverConfigs.reverse.map(s => TestUtils.createServer(s)) // create the topic TestUtils.createTopic(zkClient, topic, partitionReplicaAssignment = expectedReplicaAssignment, servers = servers) diff --git a/core/src/test/scala/unit/kafka/admin/DeleteTopicTest.scala b/core/src/test/scala/unit/kafka/admin/DeleteTopicTest.scala index 5d3c57a..1b3c04e 100644 --- a/core/src/test/scala/unit/kafka/admin/DeleteTopicTest.scala +++ b/core/src/test/scala/unit/kafka/admin/DeleteTopicTest.scala @@ -101,7 +101,7 @@ class DeleteTopicTest extends JUnit3Suite with ZooKeeperTestHarness { val expectedReplicaAssignment = Map(0 -> List(0, 1, 2)) val topic = "test" val topicAndPartition = TopicAndPartition(topic, 0) - val brokerConfigs = TestUtils.createBrokerConfigs(4, false) + val brokerConfigs = TestUtils.createBrokerConfigs(4) brokerConfigs.foreach(p => p.setProperty("delete.topic.enable", "true")) // create brokers val allServers = brokerConfigs.map(b => TestUtils.createServer(new KafkaConfig(b))) @@ -258,7 +258,7 @@ class DeleteTopicTest extends JUnit3Suite with ZooKeeperTestHarness { private def createTestTopicAndCluster(topic: String): Seq[KafkaServer] = { val expectedReplicaAssignment = Map(0 -> List(0, 1, 2)) val topicAndPartition = TopicAndPartition(topic, 0) - val brokerConfigs = TestUtils.createBrokerConfigs(3, false) + val brokerConfigs = TestUtils.createBrokerConfigs(3) brokerConfigs.foreach(p => p.setProperty("delete.topic.enable", "true")) // create brokers val servers = brokerConfigs.map(b => TestUtils.createServer(new KafkaConfig(b))) diff --git a/core/src/test/scala/unit/kafka/integration/RollingBounceTest.scala b/core/src/test/scala/unit/kafka/integration/RollingBounceTest.scala index eab4b5f..5eee08a 100644 --- a/core/src/test/scala/unit/kafka/integration/RollingBounceTest.scala +++ b/core/src/test/scala/unit/kafka/integration/RollingBounceTest.scala @@ -35,11 +35,15 @@ class RollingBounceTest extends JUnit3Suite with ZooKeeperTestHarness { val port3 = TestUtils.choosePort() val port4 = TestUtils.choosePort() - // controlled.shutdown.enable is true by default + val enableShutdown = true val configProps1 = TestUtils.createBrokerConfig(brokerId1, port1) + configProps1.put("controlled.shutdown.enable", "true") val configProps2 = TestUtils.createBrokerConfig(brokerId2, port2) + configProps2.put("controlled.shutdown.enable", "true") val configProps3 = TestUtils.createBrokerConfig(brokerId3, port3) + configProps3.put("controlled.shutdown.enable", "true") val configProps4 = TestUtils.createBrokerConfig(brokerId4, port4) + configProps4.put("controlled.shutdown.enable", "true") configProps4.put("controlled.shutdown.retry.backoff.ms", "100") var servers: Seq[KafkaServer] = Seq.empty[KafkaServer] @@ -107,4 +111,4 @@ class RollingBounceTest extends JUnit3Suite with ZooKeeperTestHarness { // Start the server back up again servers(prevLeader).startup() } -} +} \ No newline at end of file diff --git a/core/src/test/scala/unit/kafka/producer/ProducerTest.scala b/core/src/test/scala/unit/kafka/producer/ProducerTest.scala index dd71d81..b61c0b8 100644 --- a/core/src/test/scala/unit/kafka/producer/ProducerTest.scala +++ b/core/src/test/scala/unit/kafka/producer/ProducerTest.scala @@ -48,10 +48,10 @@ class ProducerTest extends JUnit3Suite with ZooKeeperTestHarness with Logging{ private val requestHandlerLogger = Logger.getLogger(classOf[KafkaRequestHandler]) private var servers = List.empty[KafkaServer] - private val props1 = TestUtils.createBrokerConfig(brokerId1, port1, false) + private val props1 = TestUtils.createBrokerConfig(brokerId1, port1) props1.put("num.partitions", "4") private val config1 = new KafkaConfig(props1) - private val props2 = TestUtils.createBrokerConfig(brokerId2, port2, false) + private val props2 = TestUtils.createBrokerConfig(brokerId2, port2) props2.put("num.partitions", "4") private val config2 = new KafkaConfig(props2) @@ -314,7 +314,7 @@ class ProducerTest extends JUnit3Suite with ZooKeeperTestHarness with Logging{ // make sure we don't wait fewer than timeoutMs assertTrue((t2-t1) >= timeoutMs) } - + @Test def testSendNullMessage() { val producer = TestUtils.createProducer[String, String]( @@ -332,7 +332,7 @@ class ProducerTest extends JUnit3Suite with ZooKeeperTestHarness with Logging{ "Topic new-topic not created after timeout", waitTime = zookeeper.tickTime) TestUtils.waitUntilLeaderIsElectedOrChanged(zkClient, "new-topic", 0) - + producer.send(new KeyedMessage[String, String]("new-topic", "key", null)) } finally { producer.close() diff --git a/core/src/test/scala/unit/kafka/producer/SyncProducerTest.scala b/core/src/test/scala/unit/kafka/producer/SyncProducerTest.scala index 24deea0..0dec9ec 100644 --- a/core/src/test/scala/unit/kafka/producer/SyncProducerTest.scala +++ b/core/src/test/scala/unit/kafka/producer/SyncProducerTest.scala @@ -31,8 +31,7 @@ import kafka.common.{TopicAndPartition, ErrorMapping} class SyncProducerTest extends JUnit3Suite with KafkaServerTestHarness { private var messageBytes = new Array[Byte](2); - // turning off controlled shutdown since testProducerCanTimeout() explicitly shuts down request handler pool. - val configs = List(new KafkaConfig(TestUtils.createBrokerConfigs(1, false).head)) + val configs = List(new KafkaConfig(TestUtils.createBrokerConfigs(1).head)) val zookeeperConnect = TestZKUtils.zookeeperConnect @Test diff --git a/core/src/test/scala/unit/kafka/server/LeaderElectionTest.scala b/core/src/test/scala/unit/kafka/server/LeaderElectionTest.scala index c2ba07c..25dffcf 100644 --- a/core/src/test/scala/unit/kafka/server/LeaderElectionTest.scala +++ b/core/src/test/scala/unit/kafka/server/LeaderElectionTest.scala @@ -34,8 +34,8 @@ class LeaderElectionTest extends JUnit3Suite with ZooKeeperTestHarness { val port1 = TestUtils.choosePort() val port2 = TestUtils.choosePort() - val configProps1 = TestUtils.createBrokerConfig(brokerId1, port1, false) - val configProps2 = TestUtils.createBrokerConfig(brokerId2, port2, false) + val configProps1 = TestUtils.createBrokerConfig(brokerId1, port1) + val configProps2 = TestUtils.createBrokerConfig(brokerId2, port2) var servers: Seq[KafkaServer] = Seq.empty[KafkaServer] var staleControllerEpochDetected = false @@ -146,4 +146,4 @@ class LeaderElectionTest extends JUnit3Suite with ZooKeeperTestHarness { case _ => false } } -} +} \ No newline at end of file diff --git a/core/src/test/scala/unit/kafka/server/LogRecoveryTest.scala b/core/src/test/scala/unit/kafka/server/LogRecoveryTest.scala index 0ec120a..b349fce 100644 --- a/core/src/test/scala/unit/kafka/server/LogRecoveryTest.scala +++ b/core/src/test/scala/unit/kafka/server/LogRecoveryTest.scala @@ -31,7 +31,7 @@ import org.junit.Assert._ class LogRecoveryTest extends JUnit3Suite with ZooKeeperTestHarness { - val configs = TestUtils.createBrokerConfigs(2, false).map(new KafkaConfig(_) { + val configs = TestUtils.createBrokerConfigs(2).map(new KafkaConfig(_) { override val replicaLagTimeMaxMs = 5000L override val replicaLagMaxMessages = 10L override val replicaFetchWaitMaxMs = 1000 @@ -52,7 +52,7 @@ class LogRecoveryTest extends JUnit3Suite with ZooKeeperTestHarness { var hwFile1: OffsetCheckpoint = new OffsetCheckpoint(new File(configProps1.logDirs(0), ReplicaManager.HighWatermarkFilename)) var hwFile2: OffsetCheckpoint = new OffsetCheckpoint(new File(configProps2.logDirs(0), ReplicaManager.HighWatermarkFilename)) var servers: Seq[KafkaServer] = Seq.empty[KafkaServer] - + override def setUp() { super.setUp() @@ -131,7 +131,7 @@ class LogRecoveryTest extends JUnit3Suite with ZooKeeperTestHarness { sendMessages(1) hw += 1 - + // give some time for follower 1 to record leader HW of 60 TestUtils.waitUntilTrue(() => server2.replicaManager.getReplica(topic, 0).get.highWatermark == hw, @@ -162,7 +162,7 @@ class LogRecoveryTest extends JUnit3Suite with ZooKeeperTestHarness { sendMessages(2) var hw = 2L - + // allow some time for the follower to get the leader HW TestUtils.waitUntilTrue(() => server2.replicaManager.getReplica(topic, 0).get.highWatermark == hw, @@ -188,7 +188,7 @@ class LogRecoveryTest extends JUnit3Suite with ZooKeeperTestHarness { sendMessages(2) hw += 2 - + // allow some time for the follower to get the leader HW TestUtils.waitUntilTrue(() => server1.replicaManager.getReplica(topic, 0).get.highWatermark == hw, diff --git a/core/src/test/scala/unit/kafka/server/ReplicaFetchTest.scala b/core/src/test/scala/unit/kafka/server/ReplicaFetchTest.scala index da4bafc..3e0bc18 100644 --- a/core/src/test/scala/unit/kafka/server/ReplicaFetchTest.scala +++ b/core/src/test/scala/unit/kafka/server/ReplicaFetchTest.scala @@ -27,7 +27,7 @@ import junit.framework.Assert._ import kafka.common._ class ReplicaFetchTest extends JUnit3Suite with ZooKeeperTestHarness { - val props = createBrokerConfigs(2,false) + val props = createBrokerConfigs(2) val configs = props.map(p => new KafkaConfig(p)) var brokers: Seq[KafkaServer] = null val topic1 = "foo" @@ -73,4 +73,4 @@ class ReplicaFetchTest extends JUnit3Suite with ZooKeeperTestHarness { } waitUntilTrue(logsMatch, "Broker logs should be identical") } -} +} \ No newline at end of file diff --git a/core/src/test/scala/unit/kafka/utils/ReplicationUtilsTest.scala b/core/src/test/scala/unit/kafka/utils/ReplicationUtilsTest.scala index f364980..84e0855 100644 --- a/core/src/test/scala/unit/kafka/utils/ReplicationUtilsTest.scala +++ b/core/src/test/scala/unit/kafka/utils/ReplicationUtilsTest.scala @@ -17,22 +17,17 @@ package kafka.utils -import kafka.cluster.{Replica, Partition} import kafka.server.{ReplicaFetcherManager, KafkaConfig} -import kafka.utils.TestUtils._ +import kafka.api.LeaderAndIsr import kafka.zk.ZooKeeperTestHarness -import kafka.log.Log import kafka.common.TopicAndPartition import org.scalatest.junit.JUnit3Suite import org.junit.Assert._ import org.junit.Test -import org.I0Itec.zkclient.ZkClient import org.easymock.EasyMock -import org.apache.log4j.Logger class ReplicationUtilsTest extends JUnit3Suite with ZooKeeperTestHarness { - private val logger = Logger.getLogger(classOf[UtilsTest]) val topic = "my-topic-test" val partitionId = 0 val brokerId = 1 @@ -43,7 +38,7 @@ class ReplicationUtilsTest extends JUnit3Suite with ZooKeeperTestHarness { val topicData = Json.encode(Map("controller_epoch" -> 1, "leader" -> 1, "versions" -> 1, "leader_epoch" -> 1,"isr" -> List(1,2))) val topicDataVersionMismatch = Json.encode(Map("controller_epoch" -> 1, "leader" -> 1, - "versions" -> 2, "leader_epoch" -> 1,"isr" -> List(2,1))) + "versions" -> 2, "leader_epoch" -> 1,"isr" -> List(1,2))) val topicDataMismatch = Json.encode(Map("controller_epoch" -> 1, "leader" -> 1, "versions" -> 2, "leader_epoch" -> 2,"isr" -> List(1,2))) @@ -53,58 +48,48 @@ class ReplicationUtilsTest extends JUnit3Suite with ZooKeeperTestHarness { ZkUtils.createPersistentPath(zkClient,topicPath,topicData) } - def testCheckLeaderAndIsrZkData() { - //mismatched zkVersion with the same data - val(dataMatched1,newZkVersion1) = ReplicationUtils.checkLeaderAndIsrZkData(zkClient,topicPath,topicDataVersionMismatch,1) - assertTrue(dataMatched1) - assertEquals(newZkVersion1,0) - - //mismatched zkVersion and leaderEpoch - val(dataMatched2,newZkVersion2) = ReplicationUtils.checkLeaderAndIsrZkData(zkClient,topicPath,topicDataMismatch,1) - assertFalse(dataMatched2) - assertEquals(newZkVersion2,-1) + @Test + def testUpdateLeaderAndIsr() { + val configs = TestUtils.createBrokerConfigs(1).map(new KafkaConfig(_)) + val log = EasyMock.createMock(classOf[kafka.log.Log]) + EasyMock.expect(log.logEndOffset).andReturn(20).anyTimes() + EasyMock.expect(log) + EasyMock.replay(log) + + val logManager = EasyMock.createMock(classOf[kafka.log.LogManager]) + EasyMock.expect(logManager.getLog(TopicAndPartition(topic, partitionId))).andReturn(Some(log)).anyTimes() + EasyMock.replay(logManager) + + val replicaManager = EasyMock.createMock(classOf[kafka.server.ReplicaManager]) + EasyMock.expect(replicaManager.config).andReturn(configs.head) + EasyMock.expect(replicaManager.logManager).andReturn(logManager) + EasyMock.expect(replicaManager.replicaFetcherManager).andReturn(EasyMock.createMock(classOf[ReplicaFetcherManager])) + EasyMock.expect(replicaManager.zkClient).andReturn(zkClient) + EasyMock.replay(replicaManager) + + val replicas = List(0,1) + + // regular update + val newLeaderAndIsr1 = new LeaderAndIsr(brokerId, leaderEpoch, replicas, 0) + val (updateSucceeded1,newZkVersion1) = ReplicationUtils.updateLeaderAndIsr(zkClient, + "my-topic-test", partitionId, newLeaderAndIsr1, controllerEpoch, 0) + assertTrue(updateSucceeded1) + assertEquals(newZkVersion1, 1) + + // mismatched zkVersion with the same data + val newLeaderAndIsr2 = new LeaderAndIsr(brokerId, leaderEpoch, replicas, zkVersion + 1) + val (updateSucceeded2,newZkVersion2) = ReplicationUtils.updateLeaderAndIsr(zkClient, + "my-topic-test", partitionId, newLeaderAndIsr2, controllerEpoch, zkVersion + 1) + assertTrue(updateSucceeded2) + // returns true with existing zkVersion + assertEquals(newZkVersion2,1) + + // mismatched zkVersion and leaderEpoch + val newLeaderAndIsr3 = new LeaderAndIsr(brokerId, leaderEpoch + 1, replicas, zkVersion + 1) + val (updateSucceeded3,newZkVersion3) = ReplicationUtils.updateLeaderAndIsr(zkClient, + "my-topic-test", partitionId, newLeaderAndIsr3, controllerEpoch, zkVersion + 1) + assertFalse(updateSucceeded3) + assertEquals(newZkVersion3,-1) } - def testUpdateIsr() { - val configs = TestUtils.createBrokerConfigs(1).map(new KafkaConfig(_)) - - val log = EasyMock.createMock(classOf[kafka.log.Log]) - EasyMock.expect(log.logEndOffset).andReturn(20).anyTimes() - EasyMock.expect(log) - EasyMock.replay(log) - - val logManager = EasyMock.createMock(classOf[kafka.log.LogManager]) - EasyMock.expect(logManager.getLog(TopicAndPartition(topic, partitionId))).andReturn(Some(log)).anyTimes() - EasyMock.replay(logManager) - - val replicaManager = EasyMock.createMock(classOf[kafka.server.ReplicaManager]) - EasyMock.expect(replicaManager.config).andReturn(configs.head) - EasyMock.expect(replicaManager.logManager).andReturn(logManager) - EasyMock.expect(replicaManager.replicaFetcherManager).andReturn(EasyMock.createMock(classOf[ReplicaFetcherManager])) - EasyMock.expect(replicaManager.zkClient).andReturn(zkClient) - EasyMock.replay(replicaManager) - - val partition = new Partition(topic,0,1,new MockTime,replicaManager) - val replicas = Set(new Replica(1,partition),new Replica(2,partition)) - - // regular update - val (updateSucceeded1,newZkVersion1) = ReplicationUtils.updateIsr(zkClient, - "my-topic-test", partitionId, brokerId, leaderEpoch, controllerEpoch, 0, replicas) - assertTrue(updateSucceeded1) - assertEquals(newZkVersion1,1) - - // mismatched zkVersion with the same data - val (updateSucceeded2,newZkVersion2) = ReplicationUtils.updateIsr(zkClient, - "my-topic-test", partitionId, brokerId, leaderEpoch, controllerEpoch, zkVersion + 1, replicas) - assertTrue(updateSucceeded2) - // returns true with existing zkVersion - assertEquals(newZkVersion2,1) - - // mismatched zkVersion and leaderEpoch - val (updateSucceeded3,newZkVersion3) = ReplicationUtils.updateIsr(zkClient, - "my-topic-test", partitionId, brokerId, leaderEpoch + 1, controllerEpoch, zkVersion + 1, replicas) - assertFalse(updateSucceeded3) - assertEquals(newZkVersion3,-1) - } - } diff --git a/core/src/test/scala/unit/kafka/utils/TestUtils.scala b/core/src/test/scala/unit/kafka/utils/TestUtils.scala index 57b2bd5..4da0f2c 100644 --- a/core/src/test/scala/unit/kafka/utils/TestUtils.scala +++ b/core/src/test/scala/unit/kafka/utils/TestUtils.scala @@ -48,7 +48,7 @@ import org.apache.kafka.clients.producer.KafkaProducer * Utility functions to help with testing */ object TestUtils extends Logging { - + val IoTmpDir = System.getProperty("java.io.tmpdir") val Letters = "ABCDEFGHIJKLMNOPQRSTUVWXYZabcdefghijklmnopqrstuvwxyz" @@ -127,10 +127,9 @@ object TestUtils extends Logging { /** * Create a test config for the given node id */ - def createBrokerConfigs(numConfigs: Int, - enableControlledShutdown: Boolean = true): List[Properties] = { + def createBrokerConfigs(numConfigs: Int): List[Properties] = { for((port, node) <- choosePorts(numConfigs).zipWithIndex) - yield createBrokerConfig(node, port, enableControlledShutdown) + yield createBrokerConfig(node, port) } def getBrokerListStrFromConfigs(configs: Seq[KafkaConfig]): String = { @@ -140,8 +139,7 @@ object TestUtils extends Logging { /** * Create a test config for the given node id */ - def createBrokerConfig(nodeId: Int, port: Int = choosePort(), - enableControlledShutdown: Boolean = true): Properties = { + def createBrokerConfig(nodeId: Int, port: Int = choosePort()): Properties = { val props = new Properties props.put("broker.id", nodeId.toString) props.put("host.name", "localhost") @@ -149,7 +147,6 @@ object TestUtils extends Logging { props.put("log.dir", TestUtils.tempDir().getAbsolutePath) props.put("zookeeper.connect", TestZKUtils.zookeeperConnect) props.put("replica.socket.timeout.ms", "1500") - props.put("controlled.shutdown.enable", enableControlledShutdown.toString) props } @@ -343,7 +340,7 @@ object TestUtils extends Logging { * Create a producer with a few pre-configured properties. * If certain properties need to be overridden, they can be provided in producerProps. */ - def createProducer[K, V](brokerList: String, + def createProducer[K, V](brokerList: String, encoder: String = classOf[DefaultEncoder].getName, keyEncoder: String = classOf[DefaultEncoder].getName, partitioner: String = classOf[DefaultPartitioner].getName, @@ -379,7 +376,6 @@ object TestUtils extends Logging { producerProps.put(ProducerConfig.BUFFER_MEMORY_CONFIG, bufferSize.toString) producerProps.put(ProducerConfig.RETRIES_CONFIG, retries.toString) producerProps.put(ProducerConfig.RETRY_BACKOFF_MS_CONFIG, "1000") - producerProps.put(ProducerConfig.RECONNECT_BACKOFF_MS_CONFIG, "200") return new KafkaProducer(producerProps) } @@ -449,9 +445,9 @@ object TestUtils extends Logging { /** * Create a wired format request based on simple basic information */ - def produceRequest(topic: String, - partition: Int, - message: ByteBufferMessageSet, + def produceRequest(topic: String, + partition: Int, + message: ByteBufferMessageSet, acks: Int = SyncProducerConfig.DefaultRequiredAcks, timeout: Int = SyncProducerConfig.DefaultAckTimeoutMs, correlationId: Int = 0, @@ -459,10 +455,10 @@ object TestUtils extends Logging { produceRequestWithAcks(Seq(topic), Seq(partition), message, acks, timeout, correlationId, clientId) } - def produceRequestWithAcks(topics: Seq[String], - partitions: Seq[Int], - message: ByteBufferMessageSet, - acks: Int = SyncProducerConfig.DefaultRequiredAcks, + def produceRequestWithAcks(topics: Seq[String], + partitions: Seq[Int], + message: ByteBufferMessageSet, + acks: Int = SyncProducerConfig.DefaultRequiredAcks, timeout: Int = SyncProducerConfig.DefaultAckTimeoutMs, correlationId: Int = 0, clientId: String = SyncProducerConfig.DefaultClientId): ProducerRequest = { @@ -544,7 +540,7 @@ object TestUtils extends Logging { return leader } - + /** * Execute the given block. If it throws an assert error, retry. Repeat * until no error is thrown or the time limit ellapses @@ -558,7 +554,7 @@ object TestUtils extends Logging { return } catch { case e: AssertionFailedError => - val ellapsed = System.currentTimeMillis - startTime + val ellapsed = System.currentTimeMillis - startTime if(ellapsed > maxWaitMs) { throw e } else { @@ -635,7 +631,7 @@ object TestUtils extends Logging { leader } - + def writeNonsenseToFile(fileName: File, position: Long, size: Int) { val file = new RandomAccessFile(fileName, "rw") file.seek(position) @@ -643,7 +639,7 @@ object TestUtils extends Logging { file.writeByte(random.nextInt(255)) file.close() } - + def appendNonsenseToFile(fileName: File, size: Int) { val file = new FileOutputStream(fileName, true) for(i <- 0 until size)