From 622d90564aa821d9c6dced867562b35766751c2e Mon Sep 17 00:00:00 2001 From: Guozhang Wang Date: Thu, 12 Feb 2015 17:04:56 -0800 Subject: [PATCH 01/11] Dummy --- .../java/org/apache/kafka/clients/KafkaClient.java | 11 +- .../org/apache/kafka/clients/NetworkClient.java | 14 +- .../kafka/clients/consumer/KafkaConsumer.java | 422 ++++++--------------- .../java/org/apache/kafka/clients/MockClient.java | 5 + 4 files changed, 146 insertions(+), 306 deletions(-) diff --git a/clients/src/main/java/org/apache/kafka/clients/KafkaClient.java b/clients/src/main/java/org/apache/kafka/clients/KafkaClient.java index 8a3e55a..96ac6d0 100644 --- a/clients/src/main/java/org/apache/kafka/clients/KafkaClient.java +++ b/clients/src/main/java/org/apache/kafka/clients/KafkaClient.java @@ -54,10 +54,19 @@ public interface KafkaClient { public long connectionDelay(Node node, long now); /** + * Check if the connection of the node has failed, based on the connection state. Such connection failure are + * usually transient and can be resumed in the next {@link #ready(org.apache.kafka.common.Node, long)} } + * call, but there are cases where transient failures needs to be caught and re-acted upon. + * + * @param node the node to check + * @return true iff the connection has failed and the node is disconnected + */ + public boolean connectionFailed(Node node); + + /** * Queue up the given request for sending. Requests can only be sent on ready connections. * * @param request The request - * @param now The current time */ public void send(ClientRequest request); diff --git a/clients/src/main/java/org/apache/kafka/clients/NetworkClient.java b/clients/src/main/java/org/apache/kafka/clients/NetworkClient.java index a7fa4a9..f429502 100644 --- a/clients/src/main/java/org/apache/kafka/clients/NetworkClient.java +++ b/clients/src/main/java/org/apache/kafka/clients/NetworkClient.java @@ -133,6 +133,19 @@ public class NetworkClient implements KafkaClient { } /** + * Check if the connection of the node has failed, based on the connection state. Such connection failure are + * usually transient and can be resumed in the next {@link #ready(org.apache.kafka.common.Node, long)} } + * call, but there are cases where transient failures needs to be caught and re-acted upon. + * + * @param node the node to check + * @return true iff the connection has failed and the node is disconnected + */ + @Override + public boolean connectionFailed(Node node) { + return connectionStates.connectionState(node.id()).equals(ConnectionState.DISCONNECTED); + } + + /** * Check if the node with the given id is ready to send more requests. * * @param node The node @@ -174,7 +187,6 @@ public class NetworkClient implements KafkaClient { * Queue up the given request for sending. Requests can only be sent out to ready nodes. * * @param request The request - * @param now The current time */ @Override public void send(ClientRequest request) { diff --git a/clients/src/main/java/org/apache/kafka/clients/consumer/KafkaConsumer.java b/clients/src/main/java/org/apache/kafka/clients/consumer/KafkaConsumer.java index 67ceb75..1d20416 100644 --- a/clients/src/main/java/org/apache/kafka/clients/consumer/KafkaConsumer.java +++ b/clients/src/main/java/org/apache/kafka/clients/consumer/KafkaConsumer.java @@ -31,10 +31,10 @@ import java.util.concurrent.atomic.AtomicInteger; import org.apache.kafka.clients.ClientRequest; import org.apache.kafka.clients.ClientResponse; import org.apache.kafka.clients.ClientUtils; -import org.apache.kafka.clients.ConnectionState; import org.apache.kafka.clients.Metadata; import org.apache.kafka.clients.NetworkClient; import org.apache.kafka.clients.RequestCompletionHandler; +import org.apache.kafka.clients.consumer.internals.Coordinator; import org.apache.kafka.clients.consumer.internals.Heartbeat; import org.apache.kafka.clients.consumer.internals.SubscriptionState; import org.apache.kafka.common.Cluster; @@ -58,25 +58,15 @@ import org.apache.kafka.common.serialization.Deserializer; import org.apache.kafka.common.network.Selector; import org.apache.kafka.common.protocol.ApiKeys; import org.apache.kafka.common.protocol.Errors; -import org.apache.kafka.common.protocol.types.Struct; import org.apache.kafka.common.record.LogEntry; import org.apache.kafka.common.record.MemoryRecords; -import org.apache.kafka.common.requests.ConsumerMetadataRequest; -import org.apache.kafka.common.requests.ConsumerMetadataResponse; import org.apache.kafka.common.requests.FetchRequest; import org.apache.kafka.common.requests.FetchRequest.PartitionData; import org.apache.kafka.common.requests.FetchResponse; -import org.apache.kafka.common.requests.HeartbeatRequest; import org.apache.kafka.common.requests.HeartbeatResponse; -import org.apache.kafka.common.requests.JoinGroupRequest; -import org.apache.kafka.common.requests.JoinGroupResponse; import org.apache.kafka.common.requests.ListOffsetRequest; import org.apache.kafka.common.requests.ListOffsetResponse; -import org.apache.kafka.common.requests.OffsetCommitRequest; import org.apache.kafka.common.requests.OffsetCommitResponse; -import org.apache.kafka.common.requests.OffsetFetchRequest; -import org.apache.kafka.common.requests.OffsetFetchResponse; -import org.apache.kafka.common.requests.RequestHeader; import org.apache.kafka.common.requests.RequestSend; import org.apache.kafka.common.utils.SystemTime; import org.apache.kafka.common.utils.Time; @@ -397,16 +387,73 @@ public class KafkaConsumer implements Consumer { private final String group; private final long sessionTimeoutMs; private final long retryBackoffMs; - private final String partitionAssignmentStrategy; private final AutoOffsetResetStrategy offsetResetStrategy; private final ConsumerRebalanceCallback rebalanceCallback; private final List> records; private final boolean checkCrcs; private long lastCommitAttemptMs; - private String consumerId; private Node consumerCoordinator; private boolean closed = false; - private int generation; + + private final Coordinator coordinator; + + private class CommitOffsetCompletionHandler implements RequestCompletionHandler { + + private final Map offsets; + + public CommitOffsetCompletionHandler(Map offsets) { + this.offsets = offsets; + } + + @Override + public void onComplete(ClientResponse resp) { + if (resp.wasDisconnected()) { + handleDisconnect(resp, time.milliseconds()); + } else { + OffsetCommitResponse response = new OffsetCommitResponse(resp.responseBody()); + for (Map.Entry entry : response.responseData().entrySet()) { + TopicPartition tp = entry.getKey(); + short errorCode = entry.getValue(); + long offset = this.offsets.get(tp); + if (errorCode == Errors.NONE.code()) { + log.debug("Committed offset {} for partition {}", offset, tp); + subscriptions.committed(tp, offset); + } else if (errorCode == Errors.CONSUMER_COORDINATOR_NOT_AVAILABLE.code() + || errorCode == Errors.NOT_COORDINATOR_FOR_CONSUMER.code()) { + coordinatorDead(); + } else { + log.error("Error committing partition {} at offset {}: {}", + tp, + offset, + Errors.forCode(errorCode).exception().getMessage()); + } + } + } + metrics.commitLatency.record(resp.requestLatencyMs()); + } + } + + private class HeartbeatCompletionHandler implements RequestCompletionHandler { + @Override + public void onComplete(ClientResponse resp) { + if (resp.wasDisconnected()) { + coordinatorDead(); + } else { + HeartbeatResponse response = new HeartbeatResponse(resp.responseBody()); + if (response.errorCode() == Errors.NONE.code()) { + log.debug("Received successful heartbeat response."); + heartbeat.receivedResponse(time.milliseconds()); + } else if (response.errorCode() == Errors.CONSUMER_COORDINATOR_NOT_AVAILABLE.code() + || response.errorCode() == Errors.NOT_COORDINATOR_FOR_CONSUMER.code()) { + coordinatorDead(); + } else { + throw new KafkaException("Unexpected error in hearbeat response: " + + Errors.forCode(response.errorCode()).exception().getMessage()); + } + } + metrics.heartbeatLatency.record(resp.requestLatencyMs()); + } + } /** * A consumer is instantiated by providing a set of key-value pairs as configuration. Valid configuration strings @@ -533,7 +580,6 @@ public class KafkaConsumer implements Consumer { this.records = new LinkedList>(); this.sessionTimeoutMs = config.getLong(ConsumerConfig.SESSION_TIMEOUT_MS_CONFIG); this.heartbeat = new Heartbeat(this.sessionTimeoutMs, time.milliseconds()); - this.partitionAssignmentStrategy = config.getString(ConsumerConfig.PARTITION_ASSIGNMENT_STRATEGY_CONFIG); this.offsetResetStrategy = AutoOffsetResetStrategy.valueOf(config.getString(ConsumerConfig.AUTO_OFFSET_RESET_CONFIG) .toUpperCase()); this.checkCrcs = config.getBoolean(ConsumerConfig.CHECK_CRCS_CONFIG); @@ -566,18 +612,23 @@ public class KafkaConsumer implements Consumer { this.client = new NetworkClient(new Selector(metrics, time, metricsGroup, metricsTags), this.metadata, clientId, - 100, + 100, // a fixed large enough value will suffice reconnectBackoffMs, sendBuffer, receiveBuffer); this.subscriptions = new SubscriptionState(); + this.coordinator = new Coordinator(this.client, + this.subscriptions, + config.getString(ConsumerConfig.PARTITION_ASSIGNMENT_STRATEGY_CONFIG), + this.sessionTimeoutMs, + this.retryBackoffMs, + this.group, + this.time); this.metrics = new ConsumerMetrics(metrics, metricsGroup, metricsTags); config.logUnused(); this.consumerCoordinator = null; - this.consumerId = ""; - this.generation = -1; log.debug("Kafka consumer created"); } @@ -683,14 +734,40 @@ public class KafkaConsumer implements Consumer { long now = time.milliseconds(); if (subscriptions.partitionsAutoAssigned()) { - // get partition assignment if needed + // rebalance to get partition assignment if (subscriptions.needsPartitionAssignment()) { - joinGroup(now); - } else if (!heartbeat.isAlive(now)) { - log.error("Failed heartbeat check."); - coordinatorDead(); - } else if (heartbeat.shouldHeartbeat(now)) { - initiateHeartbeat(now); + // execute the user's callback before rebalance + log.debug("Revoking previously assigned partitions {}", this.subscriptions.assignedPartitions()); + + try { + // TODO: Hmmm, is passing the full Consumer like this actually safe? + // Need to think about reentrancy... + this.rebalanceCallback.onPartitionsRevoked(this, this.subscriptions.assignedPartitions()); + } catch (Exception e) { + log.error("User provided callback " + this.rebalanceCallback.getClass().getName() + + " failed on partition revocation: ", e); + } + + // get new assigned partitions from the coordinator + this.subscriptions.changePartitionAssignment(coordinator.assignPartitions( + new ArrayList(this.subscriptions.subscribedTopics()), now)); + + // execute the callback + log.debug("Revoking previously assigned partitions {}", this.subscriptions.assignedPartitions()); + + try { + // TODO: Hmmm, is passing the full Consumer like this actually safe? + this.rebalanceCallback.onPartitionsAssigned(this, this.subscriptions.assignedPartitions()); + } catch (Exception e) { + log.error("User provided callback " + this.rebalanceCallback.getClass().getName() + + " failed on partition assignment: ", e); + } + + // record re-assignment time + this.metrics.partitionReassignments.record(time.milliseconds() - now); + } else { + // try to heartbeat with the coordinator if needed + coordinator.maybeHeartbeat(new HeartbeatCompletionHandler(), now); } } @@ -737,68 +814,15 @@ public class KafkaConsumer implements Consumer { public synchronized void commit(final Map offsets, CommitType commitType) { ensureNotClosed(); log.debug("Committing offsets ({}): {} ", commitType.toString().toLowerCase(), offsets); + long now = time.milliseconds(); this.lastCommitAttemptMs = now; - if (!offsets.isEmpty()) { - Map offsetData = new HashMap(offsets.size()); - for (Map.Entry entry : offsets.entrySet()) - offsetData.put(entry.getKey(), new OffsetCommitRequest.PartitionData(entry.getValue(), now, "")); - OffsetCommitRequest req = new OffsetCommitRequest(this.group, this.generation, this.consumerId, offsetData); - RequestCompletionHandler handler = new RequestCompletionHandler() { - public void onComplete(ClientResponse resp) { - if (resp.wasDisconnected()) { - handleDisconnect(resp, time.milliseconds()); - } else { - OffsetCommitResponse response = new OffsetCommitResponse(resp.responseBody()); - for (Map.Entry entry : response.responseData().entrySet()) { - TopicPartition tp = entry.getKey(); - short errorCode = entry.getValue(); - long offset = offsets.get(tp); - if (errorCode == Errors.NONE.code()) { - log.debug("Committed offset {} for partition {}", offset, tp); - subscriptions.committed(tp, offset); - } else if (errorCode == Errors.CONSUMER_COORDINATOR_NOT_AVAILABLE.code() - || errorCode == Errors.NOT_COORDINATOR_FOR_CONSUMER.code()) { - coordinatorDead(); - } else { - log.error("Error committing partition {} at offset {}: {}", - tp, - offset, - Errors.forCode(errorCode).exception().getMessage()); - } - } - } - metrics.commitLatency.record(resp.requestLatencyMs()); - } - }; - - if (commitType == CommitType.ASYNC) { - this.initiateCoordinatorRequest(ApiKeys.OFFSET_COMMIT, req.toStruct(), handler, now); - return; - } else { - boolean done; - do { - ClientResponse response = blockingCoordinatorRequest(ApiKeys.OFFSET_COMMIT, - req.toStruct(), - handler, - now); - - // check for errors - done = true; - OffsetCommitResponse commitResponse = new OffsetCommitResponse(response.responseBody()); - for (short errorCode : commitResponse.responseData().values()) { - if (errorCode != Errors.NONE.code()) - done = false; - } - if (!done) { - log.debug("Error in offset commit, backing off for {} ms before retrying again.", - this.retryBackoffMs); - Utils.sleep(this.retryBackoffMs); - } - } while (!done); - } - } + // commit the offsets with the coordinator + coordinator.commitOffsets(offsets, + new CommitOffsetCompletionHandler(offsets), + commitType.equals(CommitType.SYNC), + now); } /** @@ -958,50 +982,6 @@ public class KafkaConsumer implements Consumer { } /* - * Send a join group request to the controller - */ - private void joinGroup(long now) { - log.debug("Joining group {}", group); - - // execute the user's callback - try { - // TODO: Hmmm, is passing the full Consumer like this actually safe? - // Need to think about reentrancy... - this.rebalanceCallback.onPartitionsRevoked(this, this.subscriptions.assignedPartitions()); - } catch (Exception e) { - log.error("User provided callback " + this.rebalanceCallback.getClass().getName() - + " failed on partition revocation: ", e); - } - - // join the group - JoinGroupRequest jgr = new JoinGroupRequest(group, - (int) this.sessionTimeoutMs, - new ArrayList(this.subscriptions.subscribedTopics()), - this.consumerId, - this.partitionAssignmentStrategy); - ClientResponse resp = this.blockingCoordinatorRequest(ApiKeys.JOIN_GROUP, jgr.toStruct(), null, now); - // process the response - JoinGroupResponse response = new JoinGroupResponse(resp.responseBody()); - log.debug("Joined group: {}", response); - Errors.forCode(response.errorCode()).maybeThrow(); - this.consumerId = response.consumerId(); - this.subscriptions.changePartitionAssignment(response.assignedPartitions()); - this.heartbeat.receivedResponse(now); - - // execute the callback - try { - // TODO: Hmmm, is passing the full Consumer like this actually safe? - this.rebalanceCallback.onPartitionsAssigned(this, this.subscriptions.assignedPartitions()); - } catch (Exception e) { - log.error("User provided callback " + this.rebalanceCallback.getClass().getName() - + " failed on partition assignment: ", e); - } - - // record re-assignment time - this.metrics.partitionReassignments.record(time.milliseconds() - now); - } - - /* * Empty the record buffer and update the consumed position. */ private Map>> consumeBufferedRecords() { @@ -1138,155 +1118,14 @@ public class KafkaConsumer implements Consumer { return new ConsumerRecord(partition.topic(), partition.partition(), offset, key, value); } - /* - * Begin sending a heartbeat to the controller but don't wait for the response - */ - private void initiateHeartbeat(long now) { - ensureCoordinatorReady(); - log.debug("Sending heartbeat to co-ordinator."); - HeartbeatRequest req = new HeartbeatRequest(this.group, this.generation, this.consumerId); - RequestSend send = new RequestSend(this.consumerCoordinator.id(), - this.client.nextRequestHeader(ApiKeys.HEARTBEAT), - req.toStruct()); - - RequestCompletionHandler handler = new RequestCompletionHandler() { - public void onComplete(ClientResponse resp) { - if (resp.wasDisconnected()) { - coordinatorDead(); - } else { - HeartbeatResponse response = new HeartbeatResponse(resp.responseBody()); - if (response.errorCode() == Errors.NONE.code()) { - log.debug("Received successful heartbeat response."); - heartbeat.receivedResponse(time.milliseconds()); - } else if (response.errorCode() == Errors.CONSUMER_COORDINATOR_NOT_AVAILABLE.code() - || response.errorCode() == Errors.NOT_COORDINATOR_FOR_CONSUMER.code()) { - coordinatorDead(); - } else { - throw new KafkaException("Unexpected error in hearbeat response: " - + Errors.forCode(response.errorCode()).exception().getMessage()); - } - } - metrics.heartbeatLatency.record(resp.requestLatencyMs()); - } - }; - this.client.send(new ClientRequest(now, true, send, handler)); - this.heartbeat.sentHeartbeat(now); - } - private void coordinatorDead() { - log.info("Marking the co-ordinator dead."); + log.info("Marking the coordinator dead."); heartbeat.markDead(); if (subscriptions.partitionsAutoAssigned()) subscriptions.clearAssignment(); this.consumerCoordinator = null; } - /* - * Initiate a request to the co-ordinator but don't wait for a response. - */ - private void initiateCoordinatorRequest(ApiKeys api, Struct request, RequestCompletionHandler handler, long now) { - log.debug("Issuing co-ordinator request: {}: {}", api, request); - ensureCoordinatorReady(); - RequestHeader header = this.client.nextRequestHeader(api); - RequestSend send = new RequestSend(this.consumerCoordinator.id(), header, request); - ClientRequest clientRequest = new ClientRequest(now, true, send, handler); - this.client.send(clientRequest); - } - - /* - * Repeatedly attempt to send a request to the co-ordinator until a response is received (retry if we are - * disconnected). Note that this means any requests sent this way must be idempotent. - * - * @return The response - */ - private ClientResponse blockingCoordinatorRequest(ApiKeys api, - Struct request, - RequestCompletionHandler handler, - long now) { - do { - initiateCoordinatorRequest(api, request, handler, now); - List responses = this.client.completeAll(consumerCoordinator.id(), now); - if (responses.size() == 0) { - throw new IllegalStateException("This should not happen."); - } else { - ClientResponse response = responses.get(responses.size() - 1); - if (response.wasDisconnected()) { - handleDisconnect(response, time.milliseconds()); - Utils.sleep(this.retryBackoffMs); - } else { - return response; - } - } - } while (true); - } - - /* - * update the current consumer co-ordinator if needed and ensure we have a ready connection to it - */ - private void ensureCoordinatorReady() { - while (true) { - if (this.consumerCoordinator == null) - discoverCoordinator(); - - while (true) { - boolean ready = this.client.ready(this.consumerCoordinator, time.milliseconds()); - if (ready) { - return; - } else { - log.debug("No connection to co-ordinator, attempting to connect."); - this.client.poll(this.retryBackoffMs, time.milliseconds()); - ConnectionState state = this.client.connectionState(this.consumerCoordinator.id()); - if (ConnectionState.DISCONNECTED.equals(state)) { - log.debug("Co-ordinator connection failed. Attempting to re-discover."); - coordinatorDead(); - break; - } - } - } - } - } - - private void discoverCoordinator() { - while (this.consumerCoordinator == null) { - log.debug("No consumer co-ordinator known, attempting to discover one."); - Node coordinator = fetchConsumerCoordinator(); - - if (coordinator == null) { - log.debug("No co-ordinator found, backing off."); - Utils.sleep(this.retryBackoffMs); - } else { - log.debug("Found consumer co-ordinator: " + coordinator); - this.consumerCoordinator = coordinator; - } - } - } - - private Node fetchConsumerCoordinator() { - // find a node to ask about the co-ordinator - Node node = this.client.leastLoadedNode(time.milliseconds()); - while (node == null || !this.client.ready(node, time.milliseconds())) { - long now = time.milliseconds(); - this.client.poll(this.retryBackoffMs, now); - node = this.client.leastLoadedNode(now); - } - - // send the metadata request and process all responses - long now = time.milliseconds(); - this.client.send(createConsumerMetadataRequest(now)); - List responses = this.client.completeAll(node.id(), now); - if (responses.isEmpty()) { - throw new IllegalStateException("This should not happen."); - } else { - ClientResponse resp = responses.get(responses.size() - 1); - if (!resp.wasDisconnected()) { - ConsumerMetadataResponse response = new ConsumerMetadataResponse(resp.responseBody()); - if (response.errorCode() == Errors.NONE.code()) - return new Node(Integer.MAX_VALUE - response.node().id(), response.node().host(), response.node().port()); - } - } - return null; - } - /** * Update our cache of committed positions and then set the fetch position to the committed position (if there is * one) or reset it using the offset reset policy the user has configured. @@ -1298,7 +1137,7 @@ public class KafkaConsumer implements Consumer { // update the set of committed offsets refreshCommittedOffsets(now, partitions); - // reset the fetch position to the committed poisition + // reset the fetch position to the committed position for (TopicPartition tp : partitions) { if (subscriptions.fetched(tp) == null) { if (subscriptions.committed(tp) == null) { @@ -1315,24 +1154,14 @@ public class KafkaConsumer implements Consumer { * Fetch the given set of partitions and update the cache of committed offsets using the result */ private void refreshCommittedOffsets(long now, Set partitions) { - log.debug("Fetching committed offsets for partitions: " + Utils.join(partitions, ", ")); - OffsetFetchRequest request = new OffsetFetchRequest(this.group, new ArrayList(partitions)); - ClientResponse resp = this.blockingCoordinatorRequest(ApiKeys.OFFSET_FETCH, request.toStruct(), null, now); - OffsetFetchResponse response = new OffsetFetchResponse(resp.responseBody()); - for (Map.Entry entry : response.responseData().entrySet()) { + + // contact coordinator to fetch committed offsets + Map offsets = coordinator.fetchOffsets(now, partitions); + + // update the position with the offsets + for (Map.Entry entry : offsets.entrySet()) { TopicPartition tp = entry.getKey(); - OffsetFetchResponse.PartitionData data = entry.getValue(); - if (data.hasError()) { - log.debug("Error fetching offset for topic-partition {}: {}", tp, Errors.forCode(data.errorCode) - .exception() - .getMessage()); - } else if (data.offset >= 0) { - // update the position with the offset (-1 seems to indicate no - // such offset known) - this.subscriptions.committed(tp, data.offset); - } else { - log.debug("No committed offset for partition " + tp); - } + this.subscriptions.committed(tp, entry.getValue()); } } @@ -1389,21 +1218,6 @@ public class KafkaConsumer implements Consumer { } } - /* - * Create a consumer metadata request for the given group - */ - private ClientRequest createConsumerMetadataRequest(long now) { - ConsumerMetadataRequest request = new ConsumerMetadataRequest(this.group); - Node destination = this.client.leastLoadedNode(now); - if (destination == null) // all nodes are blacked out - return null; - RequestSend send = new RequestSend(destination.id(), - this.client.nextRequestHeader(ApiKeys.CONSUMER_METADATA), - request.toStruct()); - ClientRequest consumerMetadataRequest = new ClientRequest(now, true, send, null); - return consumerMetadataRequest; - } - /** * Reset offsets for the given partition using the offset reset strategy * @@ -1455,7 +1269,7 @@ public class KafkaConsumer implements Consumer { } private static enum AutoOffsetResetStrategy { - LATEST, EARLIEST, NONE; + LATEST, EARLIEST, NONE } private class ConsumerMetrics { diff --git a/clients/src/test/java/org/apache/kafka/clients/MockClient.java b/clients/src/test/java/org/apache/kafka/clients/MockClient.java index 8f1a7a6..eba8d99 100644 --- a/clients/src/test/java/org/apache/kafka/clients/MockClient.java +++ b/clients/src/test/java/org/apache/kafka/clients/MockClient.java @@ -62,6 +62,11 @@ public class MockClient implements KafkaClient { return 0; } + @Override + public boolean connectionFailed(Node node) { + return false; + } + public void disconnect(Integer node) { Iterator iter = requests.iterator(); while (iter.hasNext()) { -- 1.7.12.4 From 04502ffbff07a84f90f6714c4956446c431e352d Mon Sep 17 00:00:00 2001 From: Guozhang Wang Date: Tue, 17 Feb 2015 16:45:05 -0800 Subject: [PATCH 02/11] merge heartbeat to coordinator request --- .../kafka/clients/consumer/KafkaConsumer.java | 3 +- .../clients/consumer/internals/Coordinator.java | 387 +++++++++++++++++++++ 2 files changed, 388 insertions(+), 2 deletions(-) create mode 100644 clients/src/main/java/org/apache/kafka/clients/consumer/internals/Coordinator.java diff --git a/clients/src/main/java/org/apache/kafka/clients/consumer/KafkaConsumer.java b/clients/src/main/java/org/apache/kafka/clients/consumer/KafkaConsumer.java index 1d20416..a6f0de3 100644 --- a/clients/src/main/java/org/apache/kafka/clients/consumer/KafkaConsumer.java +++ b/clients/src/main/java/org/apache/kafka/clients/consumer/KafkaConsumer.java @@ -618,7 +618,6 @@ public class KafkaConsumer implements Consumer { receiveBuffer); this.subscriptions = new SubscriptionState(); this.coordinator = new Coordinator(this.client, - this.subscriptions, config.getString(ConsumerConfig.PARTITION_ASSIGNMENT_STRATEGY_CONFIG), this.sessionTimeoutMs, this.retryBackoffMs, @@ -1156,7 +1155,7 @@ public class KafkaConsumer implements Consumer { private void refreshCommittedOffsets(long now, Set partitions) { // contact coordinator to fetch committed offsets - Map offsets = coordinator.fetchOffsets(now, partitions); + Map offsets = coordinator.fetchOffsets(partitions, now); // update the position with the offsets for (Map.Entry entry : offsets.entrySet()) { diff --git a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/Coordinator.java b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/Coordinator.java new file mode 100644 index 0000000..07957ff --- /dev/null +++ b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/Coordinator.java @@ -0,0 +1,387 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one or more contributor license agreements. See the NOTICE + * file distributed with this work for additional information regarding copyright ownership. The ASF licenses this file + * to You under the Apache License, Version 2.0 (the "License"); you may not use this file except in compliance with the + * License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software distributed under the License is distributed on + * an "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the License for the + * specific language governing permissions and limitations under the License. + */ +package org.apache.kafka.clients.consumer.internals; + +import org.apache.kafka.clients.ClientRequest; +import org.apache.kafka.clients.ClientResponse; +import org.apache.kafka.clients.KafkaClient; +import org.apache.kafka.clients.RequestCompletionHandler; +import org.apache.kafka.common.Node; +import org.apache.kafka.common.TopicPartition; +import org.apache.kafka.common.protocol.ApiKeys; +import org.apache.kafka.common.protocol.Errors; +import org.apache.kafka.common.protocol.types.Struct; +import org.apache.kafka.common.requests.*; +import org.apache.kafka.common.utils.Time; +import org.apache.kafka.common.utils.Utils; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.util.ArrayList; +import java.util.HashMap; +import java.util.List; +import java.util.Map; +import java.util.Set; + +/** + * This class manage the coordination process with kafka servers. + */ +public final class Coordinator { + + private static final Logger log = LoggerFactory.getLogger(Coordinator.class); + + private final KafkaClient client; + + private final Time time; + private final String groupId; + private final Heartbeat heartbeat; + private final long sessionTimeoutMs; + private final String assignmentStrategy; + private final long retryBackoffMs; + private Node consumerCoordinator; + private String consumerId; + private int generation; + + /** + * Initialize the coordination manager. + */ + public Coordinator(KafkaClient client, + String assignmentStrategy, + long sessionTimeoutMs, + long retryBackoffMs, + String groupId, + Time time) { + + this.time = time; + this.client = client; + this.generation = -1; + this.consumerId = ""; + this.groupId = groupId; + this.consumerCoordinator = null; + this.retryBackoffMs = retryBackoffMs; + this.sessionTimeoutMs = sessionTimeoutMs; + this.assignmentStrategy = assignmentStrategy; + this.heartbeat = new Heartbeat(this.sessionTimeoutMs, time.milliseconds()); + } + + /** + * Assign partitions for the subscribed topics + * + * @param subscribedTopics The subscribed topics list + * @param now The current time + * @return The assigned partition info + */ + public List assignPartitions(List subscribedTopics, long now) { + + // send a join group request to the coordinator + log.debug("(Re-)joining group {} with subscribed topics {}", groupId, subscribedTopics); + + JoinGroupRequest request = new JoinGroupRequest(groupId, + (int) this.sessionTimeoutMs, + subscribedTopics, + this.consumerId, + this.assignmentStrategy); + ClientResponse resp = this.blockingCoordinatorRequest(ApiKeys.JOIN_GROUP, request.toStruct(), null, now); + + // process the response + JoinGroupResponse response = new JoinGroupResponse(resp.responseBody()); + Errors.forCode(response.errorCode()).maybeThrow(); + this.consumerId = response.consumerId(); + this.heartbeat.receivedResponse(now); + + log.debug("Joined group: {}", response); + + // return assigned partitions + return response.assignedPartitions(); + } + + /** + * Commit offsets for the specified list of topics and partitions + * + * A non-blocking commit will attempt to commit offsets asychronously. No error will be thrown if the commit fails. + * A blocking commit will wait for a response acknowledging the commit. In the event of an error it will retry until + * the commit succeeds. + * + * @param offsets The list of offsets per partition that should be committed. + * @param handler The handler function upon commit offset response received. + * @param blocking Control whether the commit is blocking + * @param now The current time + */ + public void commitOffsets(final Map offsets, RequestCompletionHandler handler, boolean blocking, long now) { + if (!offsets.isEmpty()) { + // create the offset commit request + Map offsetData; + offsetData = new HashMap(offsets.size()); + for (Map.Entry entry : offsets.entrySet()) + offsetData.put(entry.getKey(), new OffsetCommitRequest.PartitionData(entry.getValue(), now, "")); + OffsetCommitRequest req = new OffsetCommitRequest(this.groupId, this.generation, this.consumerId, offsetData); + + // send request and possibly wait for response if it is blocking + if (blocking) { + boolean done; + do { + ClientResponse response = blockingCoordinatorRequest(ApiKeys.OFFSET_COMMIT, req.toStruct(), handler, now); + + // check for errors + done = true; + OffsetCommitResponse commitResponse = new OffsetCommitResponse(response.responseBody()); + for (short errorCode : commitResponse.responseData().values()) { + if (errorCode != Errors.NONE.code()) + done = false; + } + if (!done) { + log.debug("Error in offset commit, backing off for {} ms before retrying again.", + this.retryBackoffMs); + Utils.sleep(this.retryBackoffMs); + } + } while (!done); + } else { + initiateCoordinatorRequest(ApiKeys.OFFSET_COMMIT, req.toStruct(), handler, now); + } + } + } + + /** + * Fetch the committed offsets of the given set of partitions + * + * @param partitions The list of partitions which need to ask for committed offsets + * @param now The current time + * @return The fetched offset values + */ + public Map fetchOffsets(Set partitions, long now) { + log.debug("Fetching committed offsets for partitions: " + Utils.join(partitions, ", ")); + + // construct the request + OffsetFetchRequest request = new OffsetFetchRequest(this.groupId, new ArrayList(partitions)); + + // send the request and block on waiting for response + ClientResponse resp = this.blockingCoordinatorRequest(ApiKeys.OFFSET_FETCH, request.toStruct(), null, now); + + // parse the response to get the offsets + OffsetFetchResponse response = new OffsetFetchResponse(resp.responseBody()); + Map offsets = new HashMap(response.responseData().size()); + for (Map.Entry entry : response.responseData().entrySet()) { + TopicPartition tp = entry.getKey(); + OffsetFetchResponse.PartitionData data = entry.getValue(); + if (data.hasError()) { + log.debug("Error fetching offset for topic-partition {}: {}", tp, Errors.forCode(data.errorCode) + .exception() + .getMessage()); + } else if (data.offset >= 0) { + // record the position with the offset (-1 seems to indicate no + // such offset known) + offsets.put(tp, data.offset); + } else { + log.debug("No committed offset for partition " + tp); + } + } + return offsets; + } + + /** + * Attempt to heartbeat the consumer coordinator if necessary, and check if the coordinator is still alive + * + * @param handler The handler function upon heartbeat response received + * @param now The current time + */ + public void maybeHeartbeat(RequestCompletionHandler handler, long now) { + if (heartbeat.shouldHeartbeat(now)) { + HeartbeatRequest req = new HeartbeatRequest(this.groupId, this.generation, this.consumerId); + this.client.send(initiateCoordinatorRequest(ApiKeys.HEARTBEAT, req.toStruct(), handler, now)); + this.heartbeat.sentHeartbeat(now); + } + } + + /** + * Repeatedly attempt to send a request to the coordinator until a response is received (retry if we are + * disconnected). Note that this means any requests sent this way must be idempotent. + * + * @return The response + */ + private ClientResponse blockingCoordinatorRequest(ApiKeys api, + Struct request, + RequestCompletionHandler handler, + long now) { + do { + ClientRequest coordinatorRequest = initiateCoordinatorRequest(api, request, handler, now); + ClientResponse coordinatorResponse = sendAndReceive(coordinatorRequest, now); + if (coordinatorResponse.wasDisconnected()) { + handleDisconnect(coordinatorResponse); + Utils.sleep(this.retryBackoffMs); + } else { + return coordinatorResponse; + } + } while (true); + } + + /** + * Ensure the consumer coordinator is known and we have a ready connection to it + */ + private void ensureCoordinatorReady() { + while (true) { + if (this.consumerCoordinator == null) + discoverCoordinator(); + + while (true) { + boolean ready = this.client.ready(this.consumerCoordinator, time.milliseconds()); + if (ready) { + return; + } else { + log.debug("No connection to coordinator, attempting to connect."); + this.client.poll(this.retryBackoffMs, time.milliseconds()); + + // if the coordinator connection has failed, we need to + // break the inner loop to re-discover the coordinator + if (this.client.connectionFailed(this.consumerCoordinator)) { + log.debug("Coordinator connection failed. Attempting to re-discover."); + coordinatorDead(); + break; + } + } + } + } + } + + /** + * Mark the current coordinator as dead + */ + private void coordinatorDead() { + log.info("Marking the coordinator {} dead.", this.consumerCoordinator.id()); + this.heartbeat.markDead(); + this.consumerCoordinator = null; + } + + /** + * Keep discovering the consumer coordinator until it is found + */ + private void discoverCoordinator() { + while (this.consumerCoordinator == null) { + log.debug("No consumer coordinator known, attempting to discover one."); + Node coordinator = fetchConsumerCoordinator(); + + if (coordinator == null) { + log.debug("No coordinator found, backing off."); + Utils.sleep(this.retryBackoffMs); + } else { + log.debug("Found consumer coordinator: " + coordinator); + this.consumerCoordinator = coordinator; + } + } + } + + /** + * Get the current consumer coordinator information via consumer metadata request + * + * @return the consumer coordinator node + */ + private Node fetchConsumerCoordinator() { + + // initiate the consumer metadata request + ClientRequest request = initiateConsumerMetadataRequest(); + + // send the request and wait for its response + ClientResponse response = sendAndReceive(request, request.createdTime()); + + // parse the response to get the coordinator info if it is not disconnected + if (!response.wasDisconnected()) { + ConsumerMetadataResponse consumerMetadataResponse = new ConsumerMetadataResponse(response.responseBody()); + // use MAX_VALUE - node.id as the coordinator id to mimic separate connections + // for the coordinator in the underlying network client layer + // TODO: this needs to be better handled in KAFKA-1935 + if (consumerMetadataResponse.errorCode() == Errors.NONE.code()) + return new Node(Integer.MAX_VALUE - consumerMetadataResponse.node().id(), + consumerMetadataResponse.node().host(), + consumerMetadataResponse.node().port()); + } + + return null; + } + + /** + * Handle the case when the request gets cancelled due to node disconnection + */ + private void handleDisconnect(ClientResponse response) { + int correlation = response.request().request().header().correlationId(); + log.debug("Cancelled request {} with correlation id {} due to node {} being disconnected", + response.request(), + correlation, + response.request().request().destination()); + // if the disconnection happens on the coordinator, mark it as dead + if (this.consumerCoordinator != null + && response.request().request().destination() == this.consumerCoordinator.id()) + coordinatorDead(); + } + + /** + * Initiate a consumer metadata request to the least loaded node. + * + * @return The created request + */ + private ClientRequest initiateConsumerMetadataRequest() { + + // find a node to ask about the coordinator + Node node = this.client.leastLoadedNode(time.milliseconds()); + while (node == null || !this.client.ready(node, time.milliseconds())) { + long now = time.milliseconds(); + this.client.poll(this.retryBackoffMs, now); + node = this.client.leastLoadedNode(now); + } + + // create a consumer metadata request + log.debug("Issuing consumer metadata request to broker {}", node.id()); + + ConsumerMetadataRequest request = new ConsumerMetadataRequest(this.groupId); + RequestSend send = new RequestSend(node.id(), + this.client.nextRequestHeader(ApiKeys.CONSUMER_METADATA), + request.toStruct()); + long now = time.milliseconds(); + return new ClientRequest(now, true, send, null); + } + + /** + * Initiate a request to the coordinator. + */ + private ClientRequest initiateCoordinatorRequest(ApiKeys api, Struct request, RequestCompletionHandler handler, long now) { + + // first make sure the coordinator is known and ready + ensureCoordinatorReady(); + + // create the request for the coordinator + log.debug("Issuing request ({}: {}) to coordinator {}", api, request, this.consumerCoordinator.id()); + + RequestHeader header = this.client.nextRequestHeader(api); + RequestSend send = new RequestSend(this.consumerCoordinator.id(), header, request); + return new ClientRequest(now, true, send, handler); + } + + /** + * Attempt to send a request and receive its response + * + * @return The response + */ + private ClientResponse sendAndReceive(ClientRequest clientRequest, long now) { + + // send the request + this.client.send(clientRequest); + + // drain all responses from the destination node + List responses = this.client.completeAll(clientRequest.request().destination(), now); + if (responses.size() == 0) { + throw new IllegalStateException("This should not happen."); + } else { + // other requests should be handled by the callback, and + // we only care about the response of the last request + return responses.get(responses.size() - 1); + } + } +} -- 1.7.12.4 From b21f7d7e2218b0037c78b87e487be027413e5b13 Mon Sep 17 00:00:00 2001 From: Guozhang Wang Date: Thu, 19 Feb 2015 14:50:23 -0800 Subject: [PATCH 03/11] v1 --- .../apache/kafka/clients/CommonClientConfigs.java | 69 ++- .../kafka/clients/consumer/ConsumerConfig.java | 10 +- .../kafka/clients/consumer/KafkaConsumer.java | 495 +++------------------ .../clients/consumer/internals/Coordinator.java | 130 ++++-- .../clients/consumer/internals/Heartbeat.java | 4 - .../kafka/clients/producer/KafkaProducer.java | 27 +- .../kafka/clients/producer/ProducerConfig.java | 10 +- .../scala/integration/kafka/api/ConsumerTest.scala | 289 ++++++++++++ 8 files changed, 523 insertions(+), 511 deletions(-) create mode 100644 core/src/test/scala/integration/kafka/api/ConsumerTest.scala diff --git a/clients/src/main/java/org/apache/kafka/clients/CommonClientConfigs.java b/clients/src/main/java/org/apache/kafka/clients/CommonClientConfigs.java index 06fcfe6..b7556d1 100644 --- a/clients/src/main/java/org/apache/kafka/clients/CommonClientConfigs.java +++ b/clients/src/main/java/org/apache/kafka/clients/CommonClientConfigs.java @@ -13,6 +13,13 @@ package org.apache.kafka.clients; +import org.apache.kafka.common.serialization.Deserializer; +import org.apache.kafka.common.serialization.Serializer; + +import java.util.Map; +import java.util.HashMap; +import java.util.Properties; + /** * Some configurations shared by both producer and consumer */ @@ -54,5 +61,65 @@ public class CommonClientConfigs { public static final String METRIC_REPORTER_CLASSES_CONFIG = "metric.reporters"; public static final String METRIC_REPORTER_CLASSES_DOC = "A list of classes to use as metrics reporters. Implementing the MetricReporter interface allows plugging in classes that will be notified of new metric creation. The JmxReporter is always included to register JMX statistics."; - + + public static final String KEY_SERIALIZER_CLASS_CONFIG = "key.serializer"; + public static final String KEY_SERIALIZER_CLASS_DOC = "Serializer class for key that implements the Serializer interface."; + + public static final String VALUE_SERIALIZER_CLASS_CONFIG = "value.serializer"; + public static final String VALUE_SERIALIZER_CLASS_DOC = "Serializer class for value that implements the Serializer interface."; + + public static final String KEY_DESERIALIZER_CLASS_CONFIG = "key.deserializer"; + public static final String KEY_DESERIALIZER_CLASS_DOC = "Deserializer class for key that implements the Deserializer interface."; + + public static final String VALUE_DESERIALIZER_CLASS_CONFIG = "value.deserializer"; + public static final String VALUE_DESERIALIZER_CLASS_DOC = "Deserializer class for value that implements the Deserializer interface."; + + + public static Map addSerializerToConfig(Map configs, + Serializer keySerializer, Serializer valueSerializer) { + Map newConfigs = new HashMap(); + newConfigs.putAll(configs); + if (keySerializer != null) + newConfigs.put(KEY_SERIALIZER_CLASS_CONFIG, keySerializer.getClass()); + if (valueSerializer != null) + newConfigs.put(VALUE_SERIALIZER_CLASS_CONFIG, valueSerializer.getClass()); + return newConfigs; + } + + public static Properties addSerializerToConfig(Properties properties, + Serializer keySerializer, Serializer valueSerializer) { + Properties newProperties = new Properties(); + newProperties.putAll(properties); + if (keySerializer != null) + newProperties.put(KEY_SERIALIZER_CLASS_CONFIG, keySerializer.getClass().getName()); + if (valueSerializer != null) + newProperties.put(VALUE_SERIALIZER_CLASS_CONFIG, valueSerializer.getClass().getName()); + return newProperties; + } + + public static Map addDeserializerToConfig(Map configs, + Deserializer keyDeserializer, + Deserializer valueDeserializer) { + Map newConfigs = new HashMap(); + newConfigs.putAll(configs); + if (keyDeserializer != null) + newConfigs.put(KEY_DESERIALIZER_CLASS_CONFIG, keyDeserializer.getClass()); + if (keyDeserializer != null) + newConfigs.put(VALUE_DESERIALIZER_CLASS_CONFIG, valueDeserializer.getClass()); + return newConfigs; + } + + public static Properties addDeserializerToConfig(Properties properties, + Deserializer keyDeserializer, + Deserializer valueDeserializer) { + Properties newProperties = new Properties(); + newProperties.putAll(properties); + if (keyDeserializer != null) + newProperties.put(KEY_DESERIALIZER_CLASS_CONFIG, keyDeserializer.getClass().getName()); + if (keyDeserializer != null) + newProperties.put(VALUE_DESERIALIZER_CLASS_CONFIG, valueDeserializer.getClass().getName()); + return newProperties; + } + + } \ No newline at end of file diff --git a/clients/src/main/java/org/apache/kafka/clients/consumer/ConsumerConfig.java b/clients/src/main/java/org/apache/kafka/clients/consumer/ConsumerConfig.java index 5fb2100..6bea5e5 100644 --- a/clients/src/main/java/org/apache/kafka/clients/consumer/ConsumerConfig.java +++ b/clients/src/main/java/org/apache/kafka/clients/consumer/ConsumerConfig.java @@ -146,12 +146,10 @@ public class ConsumerConfig extends AbstractConfig { private static final String CHECK_CRCS_DOC = "Automatically check the CRC32 of the records consumed. This ensures no on-the-wire or on-disk corruption to the messages occurred. This check adds some overhead, so it may be disabled in cases seeking extreme performance."; /** key.deserializer */ - public static final String KEY_DESERIALIZER_CLASS_CONFIG = "key.deserializer"; - private static final String KEY_DESERIALIZER_CLASS_DOC = "Deserializer class for key that implements the Deserializer interface."; + public static final String KEY_DESERIALIZER_CLASS_CONFIG = CommonClientConfigs.KEY_DESERIALIZER_CLASS_CONFIG; /** value.deserializer */ - public static final String VALUE_DESERIALIZER_CLASS_CONFIG = "value.deserializer"; - private static final String VALUE_DESERIALIZER_CLASS_DOC = "Deserializer class for value that implements the Deserializer interface."; + public static final String VALUE_DESERIALIZER_CLASS_CONFIG = CommonClientConfigs.VALUE_DESERIALIZER_CLASS_CONFIG; static { CONFIG = new ConfigDef().define(BOOTSTRAP_SERVERS_CONFIG, @@ -269,11 +267,11 @@ public class ConsumerConfig extends AbstractConfig { .define(KEY_DESERIALIZER_CLASS_CONFIG, Type.CLASS, Importance.HIGH, - KEY_DESERIALIZER_CLASS_DOC) + CommonClientConfigs.KEY_DESERIALIZER_CLASS_DOC) .define(VALUE_DESERIALIZER_CLASS_CONFIG, Type.CLASS, Importance.HIGH, - VALUE_DESERIALIZER_CLASS_DOC); + CommonClientConfigs.VALUE_DESERIALIZER_CLASS_DOC); } ConsumerConfig(Map props) { diff --git a/clients/src/main/java/org/apache/kafka/clients/consumer/KafkaConsumer.java b/clients/src/main/java/org/apache/kafka/clients/consumer/KafkaConsumer.java index a6f0de3..81c3b5f 100644 --- a/clients/src/main/java/org/apache/kafka/clients/consumer/KafkaConsumer.java +++ b/clients/src/main/java/org/apache/kafka/clients/consumer/KafkaConsumer.java @@ -13,14 +13,11 @@ package org.apache.kafka.clients.consumer; import java.net.InetSocketAddress; -import java.nio.ByteBuffer; import java.util.ArrayList; import java.util.Arrays; import java.util.Collection; import java.util.Collections; -import java.util.HashMap; import java.util.LinkedHashMap; -import java.util.LinkedList; import java.util.List; import java.util.Map; import java.util.Properties; @@ -28,19 +25,16 @@ import java.util.Set; import java.util.concurrent.TimeUnit; import java.util.concurrent.atomic.AtomicInteger; -import org.apache.kafka.clients.ClientRequest; -import org.apache.kafka.clients.ClientResponse; import org.apache.kafka.clients.ClientUtils; +import org.apache.kafka.clients.CommonClientConfigs; import org.apache.kafka.clients.Metadata; import org.apache.kafka.clients.NetworkClient; -import org.apache.kafka.clients.RequestCompletionHandler; import org.apache.kafka.clients.consumer.internals.Coordinator; +import org.apache.kafka.clients.consumer.internals.FetchManager; import org.apache.kafka.clients.consumer.internals.Heartbeat; import org.apache.kafka.clients.consumer.internals.SubscriptionState; import org.apache.kafka.common.Cluster; -import org.apache.kafka.common.KafkaException; import org.apache.kafka.common.Metric; -import org.apache.kafka.common.Node; import org.apache.kafka.common.PartitionInfo; import org.apache.kafka.common.TopicPartition; import org.apache.kafka.common.metrics.JmxReporter; @@ -56,18 +50,6 @@ import org.apache.kafka.common.metrics.stats.Max; import org.apache.kafka.common.metrics.stats.Rate; import org.apache.kafka.common.serialization.Deserializer; import org.apache.kafka.common.network.Selector; -import org.apache.kafka.common.protocol.ApiKeys; -import org.apache.kafka.common.protocol.Errors; -import org.apache.kafka.common.record.LogEntry; -import org.apache.kafka.common.record.MemoryRecords; -import org.apache.kafka.common.requests.FetchRequest; -import org.apache.kafka.common.requests.FetchRequest.PartitionData; -import org.apache.kafka.common.requests.FetchResponse; -import org.apache.kafka.common.requests.HeartbeatResponse; -import org.apache.kafka.common.requests.ListOffsetRequest; -import org.apache.kafka.common.requests.ListOffsetResponse; -import org.apache.kafka.common.requests.OffsetCommitResponse; -import org.apache.kafka.common.requests.RequestSend; import org.apache.kafka.common.utils.SystemTime; import org.apache.kafka.common.utils.Time; import org.apache.kafka.common.utils.Utils; @@ -371,90 +353,22 @@ public class KafkaConsumer implements Consumer { private static final long LATEST_OFFSET_TIMESTAMP = -1L; private static final AtomicInteger CONSUMER_CLIENT_ID_SEQUENCE = new AtomicInteger(1); + private final Coordinator coordinator; + private final FetchManager fetcher; + private final Time time; + private final NetworkClient client; private final ConsumerMetrics metrics; - private final Deserializer keyDeserializer; - private final Deserializer valueDeserializer; private final SubscriptionState subscriptions; private final Metadata metadata; private final Heartbeat heartbeat; - private final NetworkClient client; - private final int maxWaitMs; - private final int minBytes; - private final int fetchSize; + private final long retryBackoffMs; private final boolean autoCommit; private final long autoCommitIntervalMs; - private final String group; - private final long sessionTimeoutMs; - private final long retryBackoffMs; - private final AutoOffsetResetStrategy offsetResetStrategy; private final ConsumerRebalanceCallback rebalanceCallback; - private final List> records; - private final boolean checkCrcs; private long lastCommitAttemptMs; - private Node consumerCoordinator; private boolean closed = false; - private final Coordinator coordinator; - - private class CommitOffsetCompletionHandler implements RequestCompletionHandler { - - private final Map offsets; - - public CommitOffsetCompletionHandler(Map offsets) { - this.offsets = offsets; - } - - @Override - public void onComplete(ClientResponse resp) { - if (resp.wasDisconnected()) { - handleDisconnect(resp, time.milliseconds()); - } else { - OffsetCommitResponse response = new OffsetCommitResponse(resp.responseBody()); - for (Map.Entry entry : response.responseData().entrySet()) { - TopicPartition tp = entry.getKey(); - short errorCode = entry.getValue(); - long offset = this.offsets.get(tp); - if (errorCode == Errors.NONE.code()) { - log.debug("Committed offset {} for partition {}", offset, tp); - subscriptions.committed(tp, offset); - } else if (errorCode == Errors.CONSUMER_COORDINATOR_NOT_AVAILABLE.code() - || errorCode == Errors.NOT_COORDINATOR_FOR_CONSUMER.code()) { - coordinatorDead(); - } else { - log.error("Error committing partition {} at offset {}: {}", - tp, - offset, - Errors.forCode(errorCode).exception().getMessage()); - } - } - } - metrics.commitLatency.record(resp.requestLatencyMs()); - } - } - - private class HeartbeatCompletionHandler implements RequestCompletionHandler { - @Override - public void onComplete(ClientResponse resp) { - if (resp.wasDisconnected()) { - coordinatorDead(); - } else { - HeartbeatResponse response = new HeartbeatResponse(resp.responseBody()); - if (response.errorCode() == Errors.NONE.code()) { - log.debug("Received successful heartbeat response."); - heartbeat.receivedResponse(time.milliseconds()); - } else if (response.errorCode() == Errors.CONSUMER_COORDINATOR_NOT_AVAILABLE.code() - || response.errorCode() == Errors.NOT_COORDINATOR_FOR_CONSUMER.code()) { - coordinatorDead(); - } else { - throw new KafkaException("Unexpected error in hearbeat response: " - + Errors.forCode(response.errorCode()).exception().getMessage()); - } - } - metrics.heartbeatLatency.record(resp.requestLatencyMs()); - } - } - /** * A consumer is instantiated by providing a set of key-value pairs as configuration. Valid configuration strings * are documented here. Values can be @@ -487,22 +401,10 @@ public class KafkaConsumer implements Consumer { ConsumerRebalanceCallback callback, Deserializer keyDeserializer, Deserializer valueDeserializer) { - this(new ConsumerConfig(addDeserializerToConfig(configs, keyDeserializer, valueDeserializer)), - callback, - keyDeserializer, - valueDeserializer); - } - - private static Map addDeserializerToConfig(Map configs, - Deserializer keyDeserializer, - Deserializer valueDeserializer) { - Map newConfigs = new HashMap(); - newConfigs.putAll(configs); - if (keyDeserializer != null) - newConfigs.put(ConsumerConfig.KEY_DESERIALIZER_CLASS_CONFIG, keyDeserializer.getClass()); - if (keyDeserializer != null) - newConfigs.put(ConsumerConfig.VALUE_DESERIALIZER_CLASS_CONFIG, valueDeserializer.getClass()); - return newConfigs; + this(new ConsumerConfig(CommonClientConfigs.addDeserializerToConfig(configs, keyDeserializer, valueDeserializer)), + callback, + keyDeserializer, + valueDeserializer); } /** @@ -533,56 +435,25 @@ public class KafkaConsumer implements Consumer { ConsumerRebalanceCallback callback, Deserializer keyDeserializer, Deserializer valueDeserializer) { - this(new ConsumerConfig(addDeserializerToConfig(properties, keyDeserializer, valueDeserializer)), + this(new ConsumerConfig(CommonClientConfigs.addDeserializerToConfig(properties, keyDeserializer, valueDeserializer)), callback, keyDeserializer, valueDeserializer); } - private static Properties addDeserializerToConfig(Properties properties, - Deserializer keyDeserializer, - Deserializer valueDeserializer) { - Properties newProperties = new Properties(); - newProperties.putAll(properties); - if (keyDeserializer != null) - newProperties.put(ConsumerConfig.KEY_DESERIALIZER_CLASS_CONFIG, keyDeserializer.getClass().getName()); - if (keyDeserializer != null) - newProperties.put(ConsumerConfig.VALUE_DESERIALIZER_CLASS_CONFIG, valueDeserializer.getClass().getName()); - return newProperties; - } - @SuppressWarnings("unchecked") private KafkaConsumer(ConsumerConfig config, ConsumerRebalanceCallback callback, Deserializer keyDeserializer, Deserializer valueDeserializer) { log.debug("Starting the Kafka consumer"); - if (keyDeserializer == null) - this.keyDeserializer = config.getConfiguredInstance(ConsumerConfig.KEY_DESERIALIZER_CLASS_CONFIG, - Deserializer.class); - else - this.keyDeserializer = keyDeserializer; - if (valueDeserializer == null) - this.valueDeserializer = config.getConfiguredInstance(ConsumerConfig.VALUE_DESERIALIZER_CLASS_CONFIG, - Deserializer.class); - else - this.valueDeserializer = valueDeserializer; if (callback == null) this.rebalanceCallback = config.getConfiguredInstance(ConsumerConfig.CONSUMER_REBALANCE_CALLBACK_CLASS_CONFIG, ConsumerRebalanceCallback.class); else this.rebalanceCallback = callback; this.time = new SystemTime(); - this.maxWaitMs = config.getInt(ConsumerConfig.FETCH_MAX_WAIT_MS_CONFIG); - this.minBytes = config.getInt(ConsumerConfig.FETCH_MIN_BYTES_CONFIG); - this.fetchSize = config.getInt(ConsumerConfig.MAX_PARTITION_FETCH_BYTES_CONFIG); - this.group = config.getString(ConsumerConfig.GROUP_ID_CONFIG); - this.records = new LinkedList>(); - this.sessionTimeoutMs = config.getLong(ConsumerConfig.SESSION_TIMEOUT_MS_CONFIG); - this.heartbeat = new Heartbeat(this.sessionTimeoutMs, time.milliseconds()); - this.offsetResetStrategy = AutoOffsetResetStrategy.valueOf(config.getString(ConsumerConfig.AUTO_OFFSET_RESET_CONFIG) - .toUpperCase()); - this.checkCrcs = config.getBoolean(ConsumerConfig.CHECK_CRCS_CONFIG); + this.heartbeat = new Heartbeat(config.getLong(ConsumerConfig.SESSION_TIMEOUT_MS_CONFIG), time.milliseconds()); this.autoCommit = config.getBoolean(ConsumerConfig.ENABLE_AUTO_COMMIT_CONFIG); this.autoCommitIntervalMs = config.getLong(ConsumerConfig.AUTO_COMMIT_INTERVAL_MS_CONFIG); this.lastCommitAttemptMs = time.milliseconds(); @@ -606,28 +477,39 @@ public class KafkaConsumer implements Consumer { String metricsGroup = "consumer"; Map metricsTags = new LinkedHashMap(); metricsTags.put("client-id", clientId); - long reconnectBackoffMs = config.getLong(ConsumerConfig.RECONNECT_BACKOFF_MS_CONFIG); - int sendBuffer = config.getInt(ConsumerConfig.SEND_BUFFER_CONFIG); - int receiveBuffer = config.getInt(ConsumerConfig.RECEIVE_BUFFER_CONFIG); this.client = new NetworkClient(new Selector(metrics, time, metricsGroup, metricsTags), this.metadata, clientId, 100, // a fixed large enough value will suffice - reconnectBackoffMs, - sendBuffer, - receiveBuffer); + config.getLong(ConsumerConfig.RECONNECT_BACKOFF_MS_CONFIG), + config.getInt(ConsumerConfig.SEND_BUFFER_CONFIG), + config.getInt(ConsumerConfig.RECEIVE_BUFFER_CONFIG)); this.subscriptions = new SubscriptionState(); + this.metrics = new ConsumerMetrics(metrics, metricsGroup, metricsTags); this.coordinator = new Coordinator(this.client, - config.getString(ConsumerConfig.PARTITION_ASSIGNMENT_STRATEGY_CONFIG), - this.sessionTimeoutMs, + config.getString(ConsumerConfig.GROUP_ID_CONFIG), this.retryBackoffMs, - this.group, + config.getLong(ConsumerConfig.SESSION_TIMEOUT_MS_CONFIG), + config.getString(ConsumerConfig.PARTITION_ASSIGNMENT_STRATEGY_CONFIG), + this.subscriptions, + this.metrics, this.time); - this.metrics = new ConsumerMetrics(metrics, metricsGroup, metricsTags); + this.fetcher = new FetchManager(this.client, + this.retryBackoffMs, + config.getInt(ConsumerConfig.FETCH_MIN_BYTES_CONFIG), + config.getInt(ConsumerConfig.FETCH_MAX_WAIT_MS_CONFIG), + config.getInt(ConsumerConfig.MAX_PARTITION_FETCH_BYTES_CONFIG), + config.getBoolean(ConsumerConfig.CHECK_CRCS_CONFIG), + config.getString(ConsumerConfig.AUTO_OFFSET_RESET_CONFIG).toUpperCase(), + keyDeserializer == null ? config.getConfiguredInstance(ConsumerConfig.KEY_DESERIALIZER_CLASS_CONFIG, Deserializer.class) : keyDeserializer, + valueDeserializer == null ? config.getConfiguredInstance(ConsumerConfig.VALUE_DESERIALIZER_CLASS_CONFIG, Deserializer.class) : valueDeserializer, + this.metadata, + this.subscriptions, + this.metrics, + this.time); config.logUnused(); - this.consumerCoordinator = null; log.debug("Kafka consumer created"); } @@ -737,10 +619,7 @@ public class KafkaConsumer implements Consumer { if (subscriptions.needsPartitionAssignment()) { // execute the user's callback before rebalance log.debug("Revoking previously assigned partitions {}", this.subscriptions.assignedPartitions()); - try { - // TODO: Hmmm, is passing the full Consumer like this actually safe? - // Need to think about reentrancy... this.rebalanceCallback.onPartitionsRevoked(this, this.subscriptions.assignedPartitions()); } catch (Exception e) { log.error("User provided callback " + this.rebalanceCallback.getClass().getName() @@ -751,11 +630,9 @@ public class KafkaConsumer implements Consumer { this.subscriptions.changePartitionAssignment(coordinator.assignPartitions( new ArrayList(this.subscriptions.subscribedTopics()), now)); - // execute the callback + // execute the user's callback after rebalance log.debug("Revoking previously assigned partitions {}", this.subscriptions.assignedPartitions()); - try { - // TODO: Hmmm, is passing the full Consumer like this actually safe? this.rebalanceCallback.onPartitionsAssigned(this, this.subscriptions.assignedPartitions()); } catch (Exception e) { log.error("User provided callback " + this.rebalanceCallback.getClass().getName() @@ -766,14 +643,14 @@ public class KafkaConsumer implements Consumer { this.metrics.partitionReassignments.record(time.milliseconds() - now); } else { // try to heartbeat with the coordinator if needed - coordinator.maybeHeartbeat(new HeartbeatCompletionHandler(), now); + coordinator.maybeHeartbeat(now); } } // fetch positions if we have partitions we're subscribed to that we // don't know the offset for if (!subscriptions.hasAllFetchPositions()) - fetchMissingPositionsOrResetThem(this.subscriptions.missingFetchPositions(), now); + fetcher.updateFetchPositions(this.subscriptions.missingFetchPositions(), now); // maybe autocommit position if (shouldAutoCommit(now)) @@ -783,16 +660,16 @@ public class KafkaConsumer implements Consumer { * initiate any needed fetches, then block for the timeout the user specified */ Cluster cluster = this.metadata.fetch(); - reinstateFetches(cluster, now); + fetcher.initFetches(cluster, now); client.poll(timeout, now); /* * initiate a fetch request for any nodes that we just got a response from without blocking */ - reinstateFetches(cluster, now); + fetcher.initFetches(cluster, now); client.poll(0, now); - return new ConsumerRecords(consumeBufferedRecords()); + return new ConsumerRecords(fetcher.fetchedRecords()); } /** @@ -818,10 +695,7 @@ public class KafkaConsumer implements Consumer { this.lastCommitAttemptMs = now; // commit the offsets with the coordinator - coordinator.commitOffsets(offsets, - new CommitOffsetCompletionHandler(offsets), - commitType.equals(CommitType.SYNC), - now); + coordinator.commitOffsets(offsets, commitType.equals(CommitType.SYNC), now); } /** @@ -860,7 +734,7 @@ public class KafkaConsumer implements Consumer { : Arrays.asList(partitions); for (TopicPartition tp : parts) { // TODO: list offset call could be optimized by grouping by node - seek(tp, listOffset(tp, EARLIEST_OFFSET_TIMESTAMP)); + seek(tp, fetcher.offsetBefore(tp, EARLIEST_OFFSET_TIMESTAMP)); } } @@ -873,7 +747,7 @@ public class KafkaConsumer implements Consumer { : Arrays.asList(partitions); for (TopicPartition tp : parts) { // TODO: list offset call could be optimized by grouping by node - seek(tp, listOffset(tp, LATEST_OFFSET_TIMESTAMP)); + seek(tp, fetcher.offsetBefore(tp, LATEST_OFFSET_TIMESTAMP)); } } @@ -891,7 +765,7 @@ public class KafkaConsumer implements Consumer { throw new IllegalArgumentException("You can only check the position for partitions assigned to this consumer."); Long offset = this.subscriptions.consumed(partition); if (offset == null) { - fetchMissingPositionsOrResetThem(Collections.singleton(partition), time.milliseconds()); + fetcher.updateFetchPositions(Collections.singleton(partition), time.milliseconds()); return this.subscriptions.consumed(partition); } else { return offset; @@ -922,7 +796,7 @@ public class KafkaConsumer implements Consumer { } else { partitionsToFetch = Collections.singleton(partition); } - this.refreshCommittedOffsets(time.milliseconds(), partitionsToFetch); + refreshCommittedOffsets(partitionsToFetch, time.milliseconds()); Long committed = this.subscriptions.committed(partition); if (committed == null) throw new NoOffsetForPartitionException("No offset has been committed for partition " + partition); @@ -980,179 +854,13 @@ public class KafkaConsumer implements Consumer { } while (this.metadata.version() == version); } - /* - * Empty the record buffer and update the consumed position. - */ - private Map>> consumeBufferedRecords() { - if (this.subscriptions.needsPartitionAssignment()) { - return Collections.emptyMap(); - } else { - Map>> drained = new HashMap>>(); - for (PartitionRecords part : this.records) { - Long consumed = subscriptions.consumed(part.partition); - if (this.subscriptions.assignedPartitions().contains(part.partition) - && (consumed == null || part.fetchOffset == consumed)) { - List> partRecs = drained.get(part.partition); - if (partRecs == null) { - partRecs = part.records; - drained.put(part.partition, partRecs); - } else { - partRecs.addAll(part.records); - } - subscriptions.consumed(part.partition, part.records.get(part.records.size() - 1).offset() + 1); - } else { - // these records aren't next in line based on the last consumed position, ignore them - // they must be from an obsolete request - log.debug("Ignoring fetched records for {} at offset {}", part.partition, part.fetchOffset); - } - } - this.records.clear(); - return drained; - } - } - - /* - * Set-up a fetch request for any node that we have assigned partitions for which doesn't have one - */ - private void reinstateFetches(Cluster cluster, long now) { - for (ClientRequest request : createFetchRequests(cluster)) { - Node node = cluster.nodeById(request.request().destination()); - if (client.ready(node, now)) { - log.trace("Initiating fetch to node {}: {}", node.id(), request); - client.send(request); - } - } - } - - /* - * Create fetch requests for all nodes for which we have assigned partitions that have no existing requests in - * flight - */ - private List createFetchRequests(Cluster cluster) { - Map> fetchable = new HashMap>(); - for (TopicPartition partition : subscriptions.assignedPartitions()) { - Node node = cluster.leaderFor(partition); - // if there is a leader and no in-flight requests, issue a new fetch - if (node != null && this.client.inFlightRequestCount(node.id()) == 0) { - Map fetch = fetchable.get(node); - if (fetch == null) { - fetch = new HashMap(); - fetchable.put(node.id(), fetch); - } - long offset = this.subscriptions.fetched(partition); - fetch.put(partition, new FetchRequest.PartitionData(offset, this.fetchSize)); - } - } - List requests = new ArrayList(fetchable.size()); - for (Map.Entry> entry : fetchable.entrySet()) { - int nodeId = entry.getKey(); - final FetchRequest fetch = new FetchRequest(this.maxWaitMs, minBytes, entry.getValue()); - RequestSend send = new RequestSend(nodeId, this.client.nextRequestHeader(ApiKeys.FETCH), fetch.toStruct()); - RequestCompletionHandler handler = new RequestCompletionHandler() { - public void onComplete(ClientResponse response) { - handleFetchResponse(response, fetch); - } - }; - requests.add(new ClientRequest(time.milliseconds(), true, send, handler)); - } - return requests; - } - - private void handleFetchResponse(ClientResponse resp, FetchRequest request) { - if (resp.wasDisconnected()) { - handleDisconnect(resp, time.milliseconds()); - } else { - int totalBytes = 0; - int totalCount = 0; - FetchResponse response = new FetchResponse(resp.responseBody()); - for (Map.Entry entry : response.responseData().entrySet()) { - TopicPartition tp = entry.getKey(); - FetchResponse.PartitionData partition = entry.getValue(); - if (!subscriptions.assignedPartitions().contains(tp)) { - log.debug("Ignoring fetched data for partition {} which is no longer assigned.", tp); - } else if (partition.errorCode == Errors.NONE.code()) { - ByteBuffer buffer = partition.recordSet; - buffer.position(buffer.limit()); // TODO: arguably we should not have to muck with the position here - MemoryRecords records = MemoryRecords.readableRecords(buffer); - long fetchOffset = request.fetchData().get(tp).offset; - int bytes = 0; - List> parsed = new ArrayList>(); - for (LogEntry logEntry : records) { - parsed.add(parseRecord(tp, logEntry)); - bytes += logEntry.size(); - } - if (parsed.size() > 0) { - ConsumerRecord record = parsed.get(parsed.size() - 1); - this.subscriptions.fetched(tp, record.offset() + 1); - this.metrics.lag.record(partition.highWatermark - record.offset()); - this.records.add(new PartitionRecords(fetchOffset, tp, parsed)); - } - this.metrics.recordTopicFetchMetrics(tp.topic(), bytes, parsed.size()); - totalBytes += bytes; - totalCount += parsed.size(); - } else if (partition.errorCode == Errors.NOT_LEADER_FOR_PARTITION.code() - || partition.errorCode == Errors.UNKNOWN_TOPIC_OR_PARTITION.code() - || partition.errorCode == Errors.LEADER_NOT_AVAILABLE.code()) { - this.metadata.requestUpdate(); - } else if (partition.errorCode == Errors.OFFSET_OUT_OF_RANGE.code()) { - // TODO: this could be optimized by grouping all out-of-range partitions - resetOffset(tp, time.milliseconds()); - } - } - this.metrics.bytesFetched.record(totalBytes); - this.metrics.recordsFetched.record(totalCount); - } - this.metrics.fetchLatency.record(resp.requestLatencyMs()); - } - - private ConsumerRecord parseRecord(TopicPartition partition, LogEntry logEntry) { - if (this.checkCrcs) - logEntry.record().ensureValid(); - long offset = logEntry.offset(); - ByteBuffer keyBytes = logEntry.record().key(); - K key = keyBytes == null ? null : this.keyDeserializer.deserialize(partition.topic(), Utils.toArray(keyBytes)); - ByteBuffer valueBytes = logEntry.record().value(); - V value = valueBytes == null ? null : this.valueDeserializer.deserialize(partition.topic(), - Utils.toArray(valueBytes)); - return new ConsumerRecord(partition.topic(), partition.partition(), offset, key, value); - } - - private void coordinatorDead() { - log.info("Marking the coordinator dead."); - heartbeat.markDead(); - if (subscriptions.partitionsAutoAssigned()) - subscriptions.clearAssignment(); - this.consumerCoordinator = null; - } - /** - * Update our cache of committed positions and then set the fetch position to the committed position (if there is - * one) or reset it using the offset reset policy the user has configured. - * - * @throws NoOffsetForPartitionException If no offset is stored for a given partition and no offset reset policy is - * defined + * Refresh the committed offsets for given set of partitions and update the cache + * + * @param partitions The list of partitions that needs refreshing committed offsets + * @param now The current time */ - private void fetchMissingPositionsOrResetThem(Set partitions, long now) { - // update the set of committed offsets - refreshCommittedOffsets(now, partitions); - - // reset the fetch position to the committed position - for (TopicPartition tp : partitions) { - if (subscriptions.fetched(tp) == null) { - if (subscriptions.committed(tp) == null) { - resetOffset(tp, now); - } else { - log.debug("Resetting offset for partition {} to committed offset"); - subscriptions.seek(tp, subscriptions.committed(tp)); - } - } - } - } - - /* - * Fetch the given set of partitions and update the cache of committed offsets using the result - */ - private void refreshCommittedOffsets(long now, Set partitions) { + private void refreshCommittedOffsets(Set partitions, long now) { // contact coordinator to fetch committed offsets Map offsets = coordinator.fetchOffsets(partitions, now); @@ -1165,89 +873,6 @@ public class KafkaConsumer implements Consumer { } /* - * Fetch a single offset before the given timestamp for the partition. - */ - private long listOffset(TopicPartition tp, long ts) { - log.debug("Fetching offsets for partition {}.", tp); - Map partitions = new HashMap(1); - partitions.put(tp, new ListOffsetRequest.PartitionData(ts, 1)); - while (true) { - long now = time.milliseconds(); - PartitionInfo info = metadata.fetch().partition(tp); - if (info == null) { - metadata.add(tp.topic()); - awaitMetadataUpdate(); - } else if (info.leader() == null) { - awaitMetadataUpdate(); - } else if (this.client.ready(info.leader(), now)) { - Node node = info.leader(); - ListOffsetRequest request = new ListOffsetRequest(-1, partitions); - RequestSend send = new RequestSend(node.id(), - this.client.nextRequestHeader(ApiKeys.LIST_OFFSETS), - request.toStruct()); - ClientRequest clientRequest = new ClientRequest(now, true, send, null); - this.client.send(clientRequest); - List responses = this.client.completeAll(node.id(), now); - if (responses.isEmpty()) - throw new IllegalStateException("This should not happen."); - ClientResponse response = responses.get(responses.size() - 1); - if (response.wasDisconnected()) { - awaitMetadataUpdate(); - } else { - ListOffsetResponse lor = new ListOffsetResponse(response.responseBody()); - short errorCode = lor.responseData().get(tp).errorCode; - if (errorCode == Errors.NONE.code()) { - List offsets = lor.responseData().get(tp).offsets; - if (offsets.size() != 1) - throw new IllegalStateException("This should not happen."); - return offsets.get(0); - } else if (errorCode == Errors.NOT_LEADER_FOR_PARTITION.code() - || errorCode == Errors.LEADER_NOT_AVAILABLE.code()) { - log.warn("Attempt to fetch offsets for partition {} failed due to obsolete leadership information, retrying.", - tp); - awaitMetadataUpdate(); - continue; - } else { - Errors.forCode(errorCode).maybeThrow(); - } - } - } else { - client.poll(this.retryBackoffMs, now); - } - } - } - - /** - * Reset offsets for the given partition using the offset reset strategy - * - * @throws NoOffsetForPartitionException If no offset reset strategy is defined - */ - private void resetOffset(TopicPartition partition, long now) { - long timestamp; - if (this.offsetResetStrategy == AutoOffsetResetStrategy.EARLIEST) - timestamp = EARLIEST_OFFSET_TIMESTAMP; - else if (this.offsetResetStrategy == AutoOffsetResetStrategy.LATEST) - timestamp = LATEST_OFFSET_TIMESTAMP; - else - throw new NoOffsetForPartitionException("No offset is set and no reset policy is defined"); - - log.debug("Resetting offset for partition {} to {} offset.", partition, this.offsetResetStrategy.name() - .toLowerCase()); - this.subscriptions.seek(partition, listOffset(partition, timestamp)); - } - - private void handleDisconnect(ClientResponse response, long now) { - int correlation = response.request().request().header().correlationId(); - log.debug("Cancelled request {} with correlation id {} due to node {} being disconnected", - response.request(), - correlation, - response.request().request().destination()); - if (this.consumerCoordinator != null - && response.request().request().destination() == this.consumerCoordinator.id()) - coordinatorDead(); - } - - /* * Check that the consumer hasn't been closed. */ private void ensureNotClosed() { @@ -1255,23 +880,7 @@ public class KafkaConsumer implements Consumer { throw new IllegalStateException("This consumer has already been closed."); } - private static class PartitionRecords { - public long fetchOffset; - public TopicPartition partition; - public List> records; - - public PartitionRecords(long fetchOffset, TopicPartition partition, List> records) { - this.fetchOffset = fetchOffset; - this.partition = partition; - this.records = records; - } - } - - private static enum AutoOffsetResetStrategy { - LATEST, EARLIEST, NONE - } - - private class ConsumerMetrics { + public class ConsumerMetrics { public final Metrics metrics; public final Sensor bytesFetched; public final Sensor recordsFetched; diff --git a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/Coordinator.java b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/Coordinator.java index 07957ff..cd7ae04 100644 --- a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/Coordinator.java +++ b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/Coordinator.java @@ -16,12 +16,25 @@ import org.apache.kafka.clients.ClientRequest; import org.apache.kafka.clients.ClientResponse; import org.apache.kafka.clients.KafkaClient; import org.apache.kafka.clients.RequestCompletionHandler; +import org.apache.kafka.clients.consumer.KafkaConsumer.ConsumerMetrics; +import org.apache.kafka.common.KafkaException; import org.apache.kafka.common.Node; import org.apache.kafka.common.TopicPartition; import org.apache.kafka.common.protocol.ApiKeys; import org.apache.kafka.common.protocol.Errors; import org.apache.kafka.common.protocol.types.Struct; -import org.apache.kafka.common.requests.*; +import org.apache.kafka.common.requests.ConsumerMetadataRequest; +import org.apache.kafka.common.requests.ConsumerMetadataResponse; +import org.apache.kafka.common.requests.HeartbeatRequest; +import org.apache.kafka.common.requests.HeartbeatResponse; +import org.apache.kafka.common.requests.JoinGroupRequest; +import org.apache.kafka.common.requests.JoinGroupResponse; +import org.apache.kafka.common.requests.OffsetCommitRequest; +import org.apache.kafka.common.requests.OffsetCommitResponse; +import org.apache.kafka.common.requests.OffsetFetchRequest; +import org.apache.kafka.common.requests.OffsetFetchResponse; +import org.apache.kafka.common.requests.RequestHeader; +import org.apache.kafka.common.requests.RequestSend; import org.apache.kafka.common.utils.Time; import org.apache.kafka.common.utils.Utils; import org.slf4j.Logger; @@ -34,40 +47,104 @@ import java.util.Map; import java.util.Set; /** - * This class manage the coordination process with kafka servers. + * This class manage the coordination process with the consumer coordinator. */ public final class Coordinator { private static final Logger log = LoggerFactory.getLogger(Coordinator.class); private final KafkaClient client; + private final ConsumerMetrics metrics; private final Time time; private final String groupId; private final Heartbeat heartbeat; private final long sessionTimeoutMs; private final String assignmentStrategy; + private final SubscriptionState subscriptions; private final long retryBackoffMs; private Node consumerCoordinator; private String consumerId; private int generation; + private class HeartbeatCompletionHandler implements RequestCompletionHandler { + @Override + public void onComplete(ClientResponse resp) { + if (resp.wasDisconnected()) { + coordinatorDead(); + } else { + HeartbeatResponse response = new HeartbeatResponse(resp.responseBody()); + if (response.errorCode() == Errors.NONE.code()) { + log.debug("Received successful heartbeat response."); + heartbeat.receivedResponse(time.milliseconds()); + } else if (response.errorCode() == Errors.CONSUMER_COORDINATOR_NOT_AVAILABLE.code() + || response.errorCode() == Errors.NOT_COORDINATOR_FOR_CONSUMER.code()) { + coordinatorDead(); + } else { + throw new KafkaException("Unexpected error in heartbeat response: " + + Errors.forCode(response.errorCode()).exception().getMessage()); + } + } + metrics.heartbeatLatency.record(resp.requestLatencyMs()); + } + } + + private class CommitOffsetCompletionHandler implements RequestCompletionHandler { + + private final Map offsets; + + public CommitOffsetCompletionHandler(Map offsets) { + this.offsets = offsets; + } + + @Override + public void onComplete(ClientResponse resp) { + if (resp.wasDisconnected()) { + handleCoordinatorDisconnect(resp); + } else { + OffsetCommitResponse response = new OffsetCommitResponse(resp.responseBody()); + for (Map.Entry entry : response.responseData().entrySet()) { + TopicPartition tp = entry.getKey(); + short errorCode = entry.getValue(); + long offset = this.offsets.get(tp); + if (errorCode == Errors.NONE.code()) { + log.debug("Committed offset {} for partition {}", offset, tp); + subscriptions.committed(tp, offset); + } else if (errorCode == Errors.CONSUMER_COORDINATOR_NOT_AVAILABLE.code() + || errorCode == Errors.NOT_COORDINATOR_FOR_CONSUMER.code()) { + coordinatorDead(); + } else { + log.error("Error committing partition {} at offset {}: {}", + tp, + offset, + Errors.forCode(errorCode).exception().getMessage()); + } + } + } + metrics.commitLatency.record(resp.requestLatencyMs()); + } + } + /** * Initialize the coordination manager. */ public Coordinator(KafkaClient client, - String assignmentStrategy, - long sessionTimeoutMs, - long retryBackoffMs, String groupId, + long retryBackoffMs, + long sessionTimeoutMs, + String assignmentStrategy, + SubscriptionState subscriptions, + ConsumerMetrics metrics, Time time) { this.time = time; this.client = client; + this.metrics = metrics; this.generation = -1; this.consumerId = ""; this.groupId = groupId; this.consumerCoordinator = null; + this.subscriptions = subscriptions; this.retryBackoffMs = retryBackoffMs; this.sessionTimeoutMs = sessionTimeoutMs; this.assignmentStrategy = assignmentStrategy; @@ -75,7 +152,7 @@ public final class Coordinator { } /** - * Assign partitions for the subscribed topics + * Assign partitions for the subscribed topics. * * @param subscribedTopics The subscribed topics list * @param now The current time @@ -106,18 +183,17 @@ public final class Coordinator { } /** - * Commit offsets for the specified list of topics and partitions + * Commit offsets for the specified list of topics and partitions. * * A non-blocking commit will attempt to commit offsets asychronously. No error will be thrown if the commit fails. * A blocking commit will wait for a response acknowledging the commit. In the event of an error it will retry until * the commit succeeds. * * @param offsets The list of offsets per partition that should be committed. - * @param handler The handler function upon commit offset response received. * @param blocking Control whether the commit is blocking * @param now The current time */ - public void commitOffsets(final Map offsets, RequestCompletionHandler handler, boolean blocking, long now) { + public void commitOffsets(final Map offsets, boolean blocking, long now) { if (!offsets.isEmpty()) { // create the offset commit request Map offsetData; @@ -127,6 +203,8 @@ public final class Coordinator { OffsetCommitRequest req = new OffsetCommitRequest(this.groupId, this.generation, this.consumerId, offsetData); // send request and possibly wait for response if it is blocking + RequestCompletionHandler handler = new CommitOffsetCompletionHandler(offsets); + if (blocking) { boolean done; do { @@ -152,7 +230,7 @@ public final class Coordinator { } /** - * Fetch the committed offsets of the given set of partitions + * Fetch the committed offsets of the given set of partitions. * * @param partitions The list of partitions which need to ask for committed offsets * @param now The current time @@ -189,15 +267,14 @@ public final class Coordinator { } /** - * Attempt to heartbeat the consumer coordinator if necessary, and check if the coordinator is still alive + * Attempt to heartbeat the consumer coordinator if necessary, and check if the coordinator is still alive. * - * @param handler The handler function upon heartbeat response received * @param now The current time */ - public void maybeHeartbeat(RequestCompletionHandler handler, long now) { + public void maybeHeartbeat(long now) { if (heartbeat.shouldHeartbeat(now)) { HeartbeatRequest req = new HeartbeatRequest(this.groupId, this.generation, this.consumerId); - this.client.send(initiateCoordinatorRequest(ApiKeys.HEARTBEAT, req.toStruct(), handler, now)); + this.client.send(initiateCoordinatorRequest(ApiKeys.HEARTBEAT, req.toStruct(), new HeartbeatCompletionHandler(), now)); this.heartbeat.sentHeartbeat(now); } } @@ -216,7 +293,7 @@ public final class Coordinator { ClientRequest coordinatorRequest = initiateCoordinatorRequest(api, request, handler, now); ClientResponse coordinatorResponse = sendAndReceive(coordinatorRequest, now); if (coordinatorResponse.wasDisconnected()) { - handleDisconnect(coordinatorResponse); + handleCoordinatorDisconnect(coordinatorResponse); Utils.sleep(this.retryBackoffMs); } else { return coordinatorResponse; @@ -225,7 +302,7 @@ public final class Coordinator { } /** - * Ensure the consumer coordinator is known and we have a ready connection to it + * Ensure the consumer coordinator is known and we have a ready connection to it. */ private void ensureCoordinatorReady() { while (true) { @@ -253,7 +330,7 @@ public final class Coordinator { } /** - * Mark the current coordinator as dead + * Mark the current coordinator as dead. */ private void coordinatorDead() { log.info("Marking the coordinator {} dead.", this.consumerCoordinator.id()); @@ -262,7 +339,7 @@ public final class Coordinator { } /** - * Keep discovering the consumer coordinator until it is found + * Keep discovering the consumer coordinator until it is found. */ private void discoverCoordinator() { while (this.consumerCoordinator == null) { @@ -280,7 +357,7 @@ public final class Coordinator { } /** - * Get the current consumer coordinator information via consumer metadata request + * Get the current consumer coordinator information via consumer metadata request. * * @return the consumer coordinator node */ @@ -308,18 +385,17 @@ public final class Coordinator { } /** - * Handle the case when the request gets cancelled due to node disconnection + * Handle the case when the request gets cancelled due to coordinator disconnection. */ - private void handleDisconnect(ClientResponse response) { + private void handleCoordinatorDisconnect(ClientResponse response) { int correlation = response.request().request().header().correlationId(); - log.debug("Cancelled request {} with correlation id {} due to node {} being disconnected", + log.debug("Cancelled request {} with correlation id {} due to coordinator {} being disconnected", response.request(), correlation, response.request().request().destination()); - // if the disconnection happens on the coordinator, mark it as dead - if (this.consumerCoordinator != null - && response.request().request().destination() == this.consumerCoordinator.id()) - coordinatorDead(); + + // mark the coordinator as dead + coordinatorDead(); } /** @@ -365,7 +441,7 @@ public final class Coordinator { } /** - * Attempt to send a request and receive its response + * Attempt to send a request and receive its response. * * @return The response */ diff --git a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/Heartbeat.java b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/Heartbeat.java index ee0751e..afa7f4b 100644 --- a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/Heartbeat.java +++ b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/Heartbeat.java @@ -45,10 +45,6 @@ public final class Heartbeat { this.lastHeartbeatResponse = -1; } - public boolean isAlive(long now) { - return now - lastHeartbeatResponse <= timeout; - } - public boolean shouldHeartbeat(long now) { return now - lastHeartbeatSend > (1.0 / HEARTBEATS_PER_SESSION_INTERVAL) * this.timeout; } 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 7397e56..b44c91d 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 @@ -20,6 +20,7 @@ import java.util.concurrent.TimeUnit; import java.util.concurrent.atomic.AtomicInteger; import org.apache.kafka.clients.ClientUtils; +import org.apache.kafka.clients.CommonClientConfigs; import org.apache.kafka.clients.Metadata; import org.apache.kafka.clients.NetworkClient; import org.apache.kafka.clients.producer.internals.Partitioner; @@ -162,21 +163,10 @@ public class KafkaProducer implements Producer { * be called in the producer when the serializer is passed in directly. */ public KafkaProducer(Map configs, Serializer keySerializer, Serializer valueSerializer) { - this(new ProducerConfig(addSerializerToConfig(configs, keySerializer, valueSerializer)), + this(new ProducerConfig(CommonClientConfigs.addSerializerToConfig(configs, keySerializer, valueSerializer)), keySerializer, valueSerializer); } - private static Map addSerializerToConfig(Map configs, - Serializer keySerializer, Serializer valueSerializer) { - Map newConfigs = new HashMap(); - newConfigs.putAll(configs); - if (keySerializer != null) - newConfigs.put(ProducerConfig.KEY_SERIALIZER_CLASS_CONFIG, keySerializer.getClass()); - if (valueSerializer != null) - newConfigs.put(ProducerConfig.VALUE_SERIALIZER_CLASS_CONFIG, valueSerializer.getClass()); - return newConfigs; - } - /** * A producer is instantiated by providing a set of key-value pairs as configuration. Valid configuration strings * are documented here. @@ -196,21 +186,10 @@ public class KafkaProducer implements Producer { * be called in the producer when the serializer is passed in directly. */ public KafkaProducer(Properties properties, Serializer keySerializer, Serializer valueSerializer) { - this(new ProducerConfig(addSerializerToConfig(properties, keySerializer, valueSerializer)), + this(new ProducerConfig(CommonClientConfigs.addSerializerToConfig(properties, keySerializer, valueSerializer)), keySerializer, valueSerializer); } - private static Properties addSerializerToConfig(Properties properties, - Serializer keySerializer, Serializer valueSerializer) { - Properties newProperties = new Properties(); - newProperties.putAll(properties); - if (keySerializer != null) - newProperties.put(ProducerConfig.KEY_SERIALIZER_CLASS_CONFIG, keySerializer.getClass().getName()); - if (valueSerializer != null) - newProperties.put(ProducerConfig.VALUE_SERIALIZER_CLASS_CONFIG, valueSerializer.getClass().getName()); - return newProperties; - } - @SuppressWarnings("unchecked") private KafkaProducer(ProducerConfig config, Serializer keySerializer, Serializer valueSerializer) { log.trace("Starting the Kafka producer"); diff --git a/clients/src/main/java/org/apache/kafka/clients/producer/ProducerConfig.java b/clients/src/main/java/org/apache/kafka/clients/producer/ProducerConfig.java index 122375c..a5fea9e 100644 --- a/clients/src/main/java/org/apache/kafka/clients/producer/ProducerConfig.java +++ b/clients/src/main/java/org/apache/kafka/clients/producer/ProducerConfig.java @@ -159,12 +159,10 @@ public class ProducerConfig extends AbstractConfig { + " message re-ordering due to retries (i.e., if retries are enabled)."; /** key.serializer */ - public static final String KEY_SERIALIZER_CLASS_CONFIG = "key.serializer"; - private static final String KEY_SERIALIZER_CLASS_DOC = "Serializer class for key that implements the Serializer interface."; + public static final String KEY_SERIALIZER_CLASS_CONFIG = CommonClientConfigs.KEY_SERIALIZER_CLASS_CONFIG; /** value.serializer */ - public static final String VALUE_SERIALIZER_CLASS_CONFIG = "value.serializer"; - private static final String VALUE_SERIALIZER_CLASS_DOC = "Serializer class for value that implements the Serializer interface."; + public static final String VALUE_SERIALIZER_CLASS_CONFIG = CommonClientConfigs.VALUE_SERIALIZER_CLASS_CONFIG; static { CONFIG = new ConfigDef().define(BOOTSTRAP_SERVERS_CONFIG, Type.LIST, Importance.HIGH, CommonClientConfigs.BOOSTRAP_SERVERS_DOC) @@ -213,8 +211,8 @@ public class ProducerConfig extends AbstractConfig { atLeast(1), Importance.LOW, MAX_IN_FLIGHT_REQUESTS_PER_CONNECTION_DOC) - .define(KEY_SERIALIZER_CLASS_CONFIG, Type.CLASS, Importance.HIGH, KEY_SERIALIZER_CLASS_DOC) - .define(VALUE_SERIALIZER_CLASS_CONFIG, Type.CLASS, Importance.HIGH, VALUE_SERIALIZER_CLASS_DOC); + .define(KEY_SERIALIZER_CLASS_CONFIG, Type.CLASS, Importance.HIGH, CommonClientConfigs.KEY_SERIALIZER_CLASS_DOC) + .define(VALUE_SERIALIZER_CLASS_CONFIG, Type.CLASS, Importance.HIGH, CommonClientConfigs.VALUE_SERIALIZER_CLASS_DOC); } ProducerConfig(Map props) { diff --git a/core/src/test/scala/integration/kafka/api/ConsumerTest.scala b/core/src/test/scala/integration/kafka/api/ConsumerTest.scala new file mode 100644 index 0000000..85c6770 --- /dev/null +++ b/core/src/test/scala/integration/kafka/api/ConsumerTest.scala @@ -0,0 +1,289 @@ +/** + * 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 kafka.api + +import org.apache.kafka.common.KafkaException +import org.apache.kafka.clients.producer.ProducerConfig +import org.apache.kafka.clients.producer.ProducerRecord +import org.apache.kafka.clients.consumer.Consumer +import org.apache.kafka.clients.consumer.KafkaConsumer +import org.apache.kafka.clients.consumer.ConsumerRebalanceCallback +import org.apache.kafka.clients.consumer.ConsumerRecord +import org.apache.kafka.clients.consumer.ConsumerConfig +import org.apache.kafka.clients.consumer.CommitType +import org.apache.kafka.common.serialization.ByteArrayDeserializer +import org.apache.kafka.common.TopicPartition +import org.apache.kafka.clients.consumer.NoOffsetForPartitionException +import scala.collection.mutable.Buffer +import scala.collection.JavaConversions._ +import java.util.ArrayList +import java.util.Arrays +import org.junit.Assert._ +import kafka.utils.TestUtils +import kafka.utils.Logging +import kafka.server.OffsetManager + +/** + * Integration tests for the new consumer that cover basic usage as well as server failures + */ +class ConsumerTest extends IntegrationTestHarness with Logging { + + val producerCount = 1 + val consumerCount = 2 + val serverCount = 3 + + val topic = "topic" + val part = 0 + val tp = new TopicPartition(topic, part) + + // configure the servers and clients + this.serverConfig.setProperty("controlled.shutdown.enable", "false") // speed up shutdown + this.serverConfig.setProperty("offsets.topic.replication.factor", "3") // don't want to lose offset + this.serverConfig.setProperty("offsets.topic.num.partitions", "1") + this.producerConfig.setProperty(ProducerConfig.ACKS_CONFIG, "all") + this.consumerConfig.setProperty(ConsumerConfig.GROUP_ID_CONFIG, "my-test") + this.consumerConfig.setProperty(ConsumerConfig.MAX_PARTITION_FETCH_BYTES_CONFIG, 4096.toString) + this.consumerConfig.setProperty(ConsumerConfig.AUTO_OFFSET_RESET_CONFIG, "earliest") + + override def setUp() { + super.setUp() + // this will trigger the creation of the consumer offsets topic + this.consumers(0).partitionsFor(OffsetManager.OffsetsTopicName) + } + + def testSimpleConsumption() { + val numRecords = 10000 + sendRecords(numRecords) + + assertEquals(0, this.consumers(0).subscriptions.size) + this.consumers(0).subscribe(tp) + assertEquals(1, this.consumers(0).subscriptions.size) + + this.consumers(0).seek(tp, 0) + consumeRecords(this.consumers(0), numRecords = numRecords, startingOffset = 0) + } + + def testAutoOffsetReset() { + sendRecords(1) + this.consumers(0).subscribe(tp) + consumeRecords(this.consumers(0), numRecords = 1, startingOffset = 0) + } + + def testSeek() { + val consumer = this.consumers(0) + val totalRecords = 50L + sendRecords(totalRecords.toInt) + consumer.subscribe(tp) + + consumer.seekToEnd(tp) + assertEquals(totalRecords, consumer.position(tp)) + assertFalse(consumer.poll(totalRecords).iterator().hasNext()) + + consumer.seekToBeginning(tp) + assertEquals(0, consumer.position(tp), 0) + consumeRecords(consumer, numRecords = 1, startingOffset = 0) + + val mid = totalRecords / 2 + consumer.seek(tp, mid) + assertEquals(mid, consumer.position(tp)) + consumeRecords(consumer, numRecords = 1, startingOffset = mid.toInt) + } + + def testGroupConsumption() { + // we need to do this test with only one server since we have the hack join group + // that just assigns the partition hosted on the local machine (with two we might get the wrong machine + this.servers.last.shutdown() + this.servers.head.shutdown() + sendRecords(10) + this.consumers(0).subscribe(topic) + consumeRecords(this.consumers(0), numRecords = 1, startingOffset = 0) + } + + def testPositionAndCommit() { + sendRecords(5) + + // committed() on a partition with no committed offset throws an exception + intercept[NoOffsetForPartitionException] { + this.consumers(0).committed(new TopicPartition(topic, 15)) + } + + // position() on a partition that we aren't subscribed to throws an exception + intercept[IllegalArgumentException] { + this.consumers(0).position(new TopicPartition(topic, 15)) + } + + this.consumers(0).subscribe(tp) + + assertEquals("position() on a partition that we are subscribed to should reset the offset", 0L, this.consumers(0).position(tp)) + this.consumers(0).commit(CommitType.SYNC) + assertEquals(0L, this.consumers(0).committed(tp)) + + consumeRecords(this.consumers(0), 5, 0) + assertEquals("After consuming 5 records, position should be 5", 5L, this.consumers(0).position(tp)) + this.consumers(0).commit(CommitType.SYNC) + assertEquals("Committed offset should be returned", 5L, this.consumers(0).committed(tp)); + + sendRecords(1) + + // another consumer in the same group should get the same position + this.consumers(1).subscribe(tp) + consumeRecords(this.consumers(1), 1, 5) + } + + def testPartitionsFor() { + val numParts = 2 + TestUtils.createTopic(this.zkClient, topic, numParts, 1, this.servers) + val parts = this.consumers(0).partitionsFor(topic) + assertNotNull(parts) + assertEquals(2, parts.length) + assertNull(this.consumers(0).partitionsFor("non-existant-topic")) + } + + def testConsumptionWithBrokerFailures() = consumeWithBrokerFailures(numRecords = 1000) + + /* + * 1. Produce a bunch of messages + * 2. Then consume the messages while killing and restarting brokers at random + */ + def consumeWithBrokerFailures(numRecords: Int) { + TestUtils.createTopic(this.zkClient, topic, 1, serverCount, this.servers) + sendRecords(numRecords) + this.producers.map(_.close) + var consumed = 0 + val consumer = this.consumers(0) + consumer.subscribe(topic) + while (consumed < numRecords) { + // check that we are getting the messages in order + for (record <- consumer.poll(200)) { + assertEquals(consumed.toLong, record.offset()) + consumed += 1 + } + consumer.commit(CommitType.SYNC); + + /* restart any dead brokers, and kill a broker (with probability 1/3) */ + restartDeadBrokers() + if (TestUtils.random.nextInt(3) == 0) { + info("Killing broker") + killRandomBroker() + } + } + } + + def testSeekAndCommitWithBrokerFailures() = seekAndCommitWithBrokerFailures(20) + + def seekAndCommitWithBrokerFailures(numIters: Int) { + // create a topic and send it some data + val numRecords = 1000 + TestUtils.createTopic(this.zkClient, topic, 1, serverCount, this.servers) + sendRecords(numRecords) + this.producers.map(_.close) + + val consumer = this.consumers(0) + consumer.subscribe(tp) + consumer.seek(tp, 0) + for (iter <- 0 until numIters) { + val coin = TestUtils.random.nextInt(4) + if (coin == 0) { + info("Seeking to end of log") + consumer.seekToEnd() + assertEquals(1000.toLong, consumer.position(tp)) + } else if (coin == 1) { + val pos = TestUtils.random.nextInt(numRecords).toLong + info("Seeking to " + pos) + consumer.seek(tp, pos) + assertEquals(pos, consumer.position(tp)) + } else if (coin == 2) { + info("Committing offset.") + consumer.commit(CommitType.SYNC) + assertEquals(consumer.position(tp), consumer.committed(tp)) + } else { + restartDeadBrokers() + killRandomBroker() + } + } + } + + def testPartitionReassignmentCallback() { + val callback = new TestConsumerReassignmentCallback() + this.consumerConfig.setProperty(ConsumerConfig.SESSION_TIMEOUT_MS_CONFIG, "200"); // timeout quickly to avoid slow test + val consumer0 = new KafkaConsumer(this.consumerConfig, callback, new ByteArrayDeserializer(), new ByteArrayDeserializer()) + consumer0.subscribe("test") + + // the initial subscription should cause a callback execution + while(callback.callsToAssigned == 0) + consumer0.poll(50) + + // get metadata for the topic + var parts = consumer0.partitionsFor(OffsetManager.OffsetsTopicName) + while(parts == null) + parts = consumer0.partitionsFor(OffsetManager.OffsetsTopicName) + assertEquals(1, parts.size) + assertNotNull(parts(0).leader()) + + // shutdown the coordinator + val coordinator = parts(0).leader().id() + this.servers(coordinator).shutdown() + + // this should cause another callback execution + while(callback.callsToAssigned < 2) + consumer0.poll(50) + assertEquals(2, callback.callsToAssigned) + assertEquals(2, callback.callsToRevoked) + + // restart the coordinator since it may also be hosting "test" topic + this.servers(coordinator).startup() + + consumer0.close() + } + + class TestConsumerReassignmentCallback extends ConsumerRebalanceCallback { + var callsToAssigned = 0 + var callsToRevoked = 0 + def onPartitionsAssigned(consumer: Consumer[_,_], partitions: java.util.Collection[TopicPartition]) { + info("onPartitionsAssigned called.") + callsToAssigned += 1 + } + def onPartitionsRevoked(consumer: Consumer[_,_], partitions: java.util.Collection[TopicPartition]) { + info("onPartitionsRevoked called.") + callsToRevoked += 1 + } + } + + private def sendRecords(numRecords: Int) { + val futures = (0 until numRecords).map { i => + this.producers(0).send(new ProducerRecord(topic, part, i.toString.getBytes, i.toString.getBytes)) + } + futures.map(_.get) + } + + private def consumeRecords(consumer: Consumer[Array[Byte], Array[Byte]], numRecords: Int, startingOffset: Int) { + val records = new ArrayList[ConsumerRecord[Array[Byte], Array[Byte]]]() + val maxIters = numRecords * 300 + var iters = 0 + while (records.size < numRecords) { + for (record <- consumer.poll(50)) + records.add(record) + if(iters > maxIters) + throw new IllegalStateException("Failed to consume the expected records after " + iters + " iterations."); + iters += 1 + } + for (i <- 0 until numRecords) { + val record = records.get(i) + val offset = startingOffset + i + assertEquals(topic, record.topic()) + assertEquals(part, record.partition()) + assertEquals(offset.toLong, record.offset()) + } + } + +} \ No newline at end of file -- 1.7.12.4 From 0f3b4d3a935c5685e386377561c7a27956035609 Mon Sep 17 00:00:00 2001 From: Guozhang Wang Date: Thu, 19 Feb 2015 14:50:43 -0800 Subject: [PATCH 04/11] add FetchManager --- .../clients/consumer/internals/FetchManager.java | 383 +++++++++++++++++++++ 1 file changed, 383 insertions(+) create mode 100644 clients/src/main/java/org/apache/kafka/clients/consumer/internals/FetchManager.java diff --git a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/FetchManager.java b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/FetchManager.java new file mode 100644 index 0000000..5beb0c2 --- /dev/null +++ b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/FetchManager.java @@ -0,0 +1,383 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one or more contributor license agreements. See the NOTICE + * file distributed with this work for additional information regarding copyright ownership. The ASF licenses this file + * to You under the Apache License, Version 2.0 (the "License"); you may not use this file except in compliance with the + * License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software distributed under the License is distributed on + * an "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the License for the + * specific language governing permissions and limitations under the License. + */ + +package org.apache.kafka.clients.consumer.internals; + +import org.apache.kafka.clients.ClientRequest; +import org.apache.kafka.clients.ClientResponse; +import org.apache.kafka.clients.KafkaClient; +import org.apache.kafka.clients.Metadata; +import org.apache.kafka.clients.RequestCompletionHandler; +import org.apache.kafka.clients.consumer.ConsumerRecord; +import org.apache.kafka.clients.consumer.KafkaConsumer.ConsumerMetrics; +import org.apache.kafka.clients.consumer.NoOffsetForPartitionException; +import org.apache.kafka.common.Cluster; +import org.apache.kafka.common.Node; +import org.apache.kafka.common.PartitionInfo; +import org.apache.kafka.common.TopicPartition; +import org.apache.kafka.common.protocol.ApiKeys; +import org.apache.kafka.common.protocol.Errors; +import org.apache.kafka.common.record.LogEntry; +import org.apache.kafka.common.record.MemoryRecords; +import org.apache.kafka.common.requests.FetchRequest; +import org.apache.kafka.common.requests.FetchResponse; +import org.apache.kafka.common.requests.ListOffsetRequest; +import org.apache.kafka.common.requests.ListOffsetResponse; +import org.apache.kafka.common.requests.RequestSend; +import org.apache.kafka.common.serialization.Deserializer; +import org.apache.kafka.common.utils.Time; +import org.apache.kafka.common.utils.Utils; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.nio.ByteBuffer; +import java.util.*; + +/** + * This class manage the fetching process with the brokers. + */ +public class FetchManager { + + private static final Logger log = LoggerFactory.getLogger(Coordinator.class); + private static final long EARLIEST_OFFSET_TIMESTAMP = -2L; + private static final long LATEST_OFFSET_TIMESTAMP = -1L; + + + private final KafkaClient client; + private final ConsumerMetrics metrics; + + private final Time time; + private final int minBytes; + private final int maxWaitMs; + private final int fetchSize; + private final boolean checkCrcs; + private final long retryBackoffMs; + private final Metadata metadata; + private final SubscriptionState subscriptions; + private final List> records; + private final AutoOffsetResetStrategy offsetResetStrategy; + private final Deserializer keyDeserializer; + private final Deserializer valueDeserializer; + + + public FetchManager(KafkaClient client, + long retryBackoffMs, + int minBytes, + int maxWaitMs, + int fetchSize, + boolean checkCrcs, + String offsetReset, + Deserializer keyDeserializer, + Deserializer valueDeserializer, + Metadata metadata, + SubscriptionState subscriptions, + ConsumerMetrics metrics, + Time time) { + + this.time = time; + this.client = client; + this.metrics = metrics; + this.metadata = metadata; + this.subscriptions = subscriptions; + this.retryBackoffMs = retryBackoffMs; + this.minBytes = minBytes; + this.maxWaitMs = maxWaitMs; + this.fetchSize = fetchSize; + this.checkCrcs = checkCrcs; + this.offsetResetStrategy = AutoOffsetResetStrategy.valueOf(offsetReset); + + this.keyDeserializer = keyDeserializer; + this.valueDeserializer = valueDeserializer; + + this.records = new LinkedList>(); + } + + /** + * Set-up a fetch request for any node that we have assigned partitions for which doesn't have one. + * + * @param cluster The current cluster metadata + * @param now The current time + */ + public void initFetches(Cluster cluster, long now) { + for (ClientRequest request : createFetchRequests(cluster)) { + Node node = cluster.nodeById(request.request().destination()); + if (client.ready(node, now)) { + log.trace("Initiating fetch to node {}: {}", node.id(), request); + client.send(request); + } + } + } + + /** + * Return the fetched records, empty the record buffer and update the consumed position. + * + * @return The fetched records per partition + */ + public Map>> fetchedRecords() { + if (this.subscriptions.needsPartitionAssignment()) { + return Collections.emptyMap(); + } else { + Map>> drained = new HashMap>>(); + for (PartitionRecords part : this.records) { + Long consumed = subscriptions.consumed(part.partition); + if (this.subscriptions.assignedPartitions().contains(part.partition) + && (consumed == null || part.fetchOffset == consumed)) { + List> records = drained.get(part.partition); + if (records == null) { + records = part.records; + drained.put(part.partition, records); + } else { + records.addAll(part.records); + } + subscriptions.consumed(part.partition, part.records.get(part.records.size() - 1).offset() + 1); + } else { + // these records aren't next in line based on the last consumed position, ignore them + // they must be from an obsolete request + log.debug("Ignoring fetched records for {} at offset {}", part.partition, part.fetchOffset); + } + } + this.records.clear(); + return drained; + } + } + + /** + * Set the fetch position to the committed position (if there is one) + * or reset it using the offset reset policy the user has configured. + * + * @param partitions The partitions that needs updating fetch positions + * @param now The current time + * @throws org.apache.kafka.clients.consumer.NoOffsetForPartitionException If no offset is stored for a given partition and no offset reset policy is + * defined + */ + public void updateFetchPositions(Set partitions, long now) { + // reset the fetch position to the committed position + for (TopicPartition tp : partitions) { + if (subscriptions.fetched(tp) == null) { + if (subscriptions.committed(tp) == null) { + resetOffset(tp, now); + } else { + log.debug("Resetting offset for partition {} to committed offset"); + subscriptions.seek(tp, subscriptions.committed(tp)); + } + } + } + } + + /** + * Fetch a single offset before the given timestamp for the partition. + * + * @param topicPartition The partition that needs fetching offset. + * @param timestamp The timestamp for fetching offset. + * @return The offset of the message that is published before the given timestamp + */ + public long offsetBefore(TopicPartition topicPartition, long timestamp) { + log.debug("Fetching offsets for partition {}.", topicPartition); + Map partitions = new HashMap(1); + partitions.put(topicPartition, new ListOffsetRequest.PartitionData(timestamp, 1)); + while (true) { + long now = time.milliseconds(); + PartitionInfo info = metadata.fetch().partition(topicPartition); + if (info == null) { + metadata.add(topicPartition.topic()); + awaitMetadataUpdate(); + } else if (info.leader() == null) { + awaitMetadataUpdate(); + } else if (this.client.ready(info.leader(), now)) { + Node node = info.leader(); + ListOffsetRequest request = new ListOffsetRequest(-1, partitions); + RequestSend send = new RequestSend(node.id(), + this.client.nextRequestHeader(ApiKeys.LIST_OFFSETS), + request.toStruct()); + ClientRequest clientRequest = new ClientRequest(now, true, send, null); + this.client.send(clientRequest); + List responses = this.client.completeAll(node.id(), now); + if (responses.isEmpty()) + throw new IllegalStateException("This should not happen."); + ClientResponse response = responses.get(responses.size() - 1); + if (response.wasDisconnected()) { + awaitMetadataUpdate(); + } else { + ListOffsetResponse lor = new ListOffsetResponse(response.responseBody()); + short errorCode = lor.responseData().get(topicPartition).errorCode; + if (errorCode == Errors.NONE.code()) { + List offsets = lor.responseData().get(topicPartition).offsets; + if (offsets.size() != 1) + throw new IllegalStateException("This should not happen."); + return offsets.get(0); + } else if (errorCode == Errors.NOT_LEADER_FOR_PARTITION.code() + || errorCode == Errors.LEADER_NOT_AVAILABLE.code()) { + log.warn("Attempt to fetch offsets for partition {} failed due to obsolete leadership information, retrying.", + topicPartition); + awaitMetadataUpdate(); + } else { + Errors.forCode(errorCode).maybeThrow(); + } + } + } else { + client.poll(this.retryBackoffMs, now); + } + } + } + + /** + * Create fetch requests for all nodes for which we have assigned partitions + * that have no existing requests in flight. + */ + private List createFetchRequests(Cluster cluster) { + // create the fetch info + Map> fetchable = new HashMap>(); + for (TopicPartition partition : subscriptions.assignedPartitions()) { + Node node = cluster.leaderFor(partition); + // if there is a leader and no in-flight requests, issue a new fetch + if (node != null && this.client.inFlightRequestCount(node.id()) == 0) { + Map fetch = fetchable.get(node); + if (fetch == null) { + fetch = new HashMap(); + fetchable.put(node.id(), fetch); + } + long offset = this.subscriptions.fetched(partition); + fetch.put(partition, new FetchRequest.PartitionData(offset, this.fetchSize)); + } + } + + // create the requests + List requests = new ArrayList(fetchable.size()); + for (Map.Entry> entry : fetchable.entrySet()) { + int nodeId = entry.getKey(); + final FetchRequest fetch = new FetchRequest(this.maxWaitMs, this.minBytes, entry.getValue()); + RequestSend send = new RequestSend(nodeId, this.client.nextRequestHeader(ApiKeys.FETCH), fetch.toStruct()); + RequestCompletionHandler handler = new RequestCompletionHandler() { + public void onComplete(ClientResponse response) { + handleFetchResponse(response, fetch); + } + }; + requests.add(new ClientRequest(time.milliseconds(), true, send, handler)); + } + return requests; + } + + /** + * The callback for fetch completion + */ + private void handleFetchResponse(ClientResponse resp, FetchRequest request) { + if (resp.wasDisconnected()) { + int correlation = resp.request().request().header().correlationId(); + log.debug("Cancelled fetch request {} with correlation id {} due to node {} being disconnected", + resp.request(), correlation, resp.request().request().destination()); + } else { + int totalBytes = 0; + int totalCount = 0; + FetchResponse response = new FetchResponse(resp.responseBody()); + for (Map.Entry entry : response.responseData().entrySet()) { + TopicPartition tp = entry.getKey(); + FetchResponse.PartitionData partition = entry.getValue(); + if (!subscriptions.assignedPartitions().contains(tp)) { + log.debug("Ignoring fetched data for partition {} which is no longer assigned.", tp); + } else if (partition.errorCode == Errors.NONE.code()) { + int bytes = 0; + ByteBuffer buffer = partition.recordSet; + buffer.position(buffer.limit()); + MemoryRecords records = MemoryRecords.readableRecords(buffer); + long fetchOffset = request.fetchData().get(tp).offset; + List> parsed = new ArrayList>(); + for (LogEntry logEntry : records) { + parsed.add(parseRecord(tp, logEntry)); + bytes += logEntry.size(); + } + if (parsed.size() > 0) { + ConsumerRecord record = parsed.get(parsed.size() - 1); + this.subscriptions.fetched(tp, record.offset() + 1); + this.records.add(new PartitionRecords(fetchOffset, tp, parsed)); + this.metrics.lag.record(partition.highWatermark - record.offset()); + } + this.metrics.recordTopicFetchMetrics(tp.topic(), bytes, parsed.size()); + totalBytes += bytes; + totalCount += parsed.size(); + } else if (partition.errorCode == Errors.NOT_LEADER_FOR_PARTITION.code() + || partition.errorCode == Errors.UNKNOWN_TOPIC_OR_PARTITION.code() + || partition.errorCode == Errors.LEADER_NOT_AVAILABLE.code()) { + this.metadata.requestUpdate(); + } else if (partition.errorCode == Errors.OFFSET_OUT_OF_RANGE.code()) { + // TODO: this could be optimized by grouping all out-of-range partitions + resetOffset(tp, time.milliseconds()); + } + } + this.metrics.bytesFetched.record(totalBytes); + this.metrics.recordsFetched.record(totalCount); + } + this.metrics.fetchLatency.record(resp.requestLatencyMs()); + } + + /** + * Parse the record entry, deserializing the key / value fields if necessary + */ + private ConsumerRecord parseRecord(TopicPartition partition, LogEntry logEntry) { + if (this.checkCrcs) + logEntry.record().ensureValid(); + + long offset = logEntry.offset(); + ByteBuffer keyBytes = logEntry.record().key(); + K key = keyBytes == null ? null : this.keyDeserializer.deserialize(partition.topic(), Utils.toArray(keyBytes)); + ByteBuffer valueBytes = logEntry.record().value(); + V value = valueBytes == null ? null : this.valueDeserializer.deserialize(partition.topic(), Utils.toArray(valueBytes)); + + return new ConsumerRecord(partition.topic(), partition.partition(), offset, key, value); + } + + /* + * Request a metadata update and wait until it has occurred + */ + private void awaitMetadataUpdate() { + int version = this.metadata.requestUpdate(); + do { + long now = time.milliseconds(); + this.client.poll(this.retryBackoffMs, now); + } while (this.metadata.version() == version); + } + + /** + * Reset offsets for the given partition using the offset reset strategy + * + * @throws org.apache.kafka.clients.consumer.NoOffsetForPartitionException If no offset reset strategy is defined + */ + private void resetOffset(TopicPartition partition, long now) { + long timestamp; + if (this.offsetResetStrategy == AutoOffsetResetStrategy.EARLIEST) + timestamp = EARLIEST_OFFSET_TIMESTAMP; + else if (this.offsetResetStrategy == AutoOffsetResetStrategy.LATEST) + timestamp = LATEST_OFFSET_TIMESTAMP; + else + throw new NoOffsetForPartitionException("No offset is set and no reset policy is defined"); + + log.debug("Resetting offset for partition {} to {} offset.", partition, this.offsetResetStrategy.name() + .toLowerCase()); + this.subscriptions.seek(partition, offsetBefore(partition, timestamp)); + } + + private static class PartitionRecords { + public long fetchOffset; + public TopicPartition partition; + public List> records; + + public PartitionRecords(long fetchOffset, TopicPartition partition, List> records) { + this.fetchOffset = fetchOffset; + this.partition = partition; + this.records = records; + } + } + + private static enum AutoOffsetResetStrategy { + LATEST, EARLIEST, NONE + } +} -- 1.7.12.4 From bf33b032ffea7e7cd2d708eb07d41e2e27f774b3 Mon Sep 17 00:00:00 2001 From: Guozhang Wang Date: Thu, 19 Feb 2015 15:59:56 -0800 Subject: [PATCH 05/11] fix one test error --- .../kafka/clients/consumer/KafkaConsumer.java | 55 +++++++++++++++++----- .../clients/consumer/internals/FetchManager.java | 54 +++++++-------------- .../consumer/internals/SubscriptionState.java | 15 +++++- 3 files changed, 73 insertions(+), 51 deletions(-) diff --git a/clients/src/main/java/org/apache/kafka/clients/consumer/KafkaConsumer.java b/clients/src/main/java/org/apache/kafka/clients/consumer/KafkaConsumer.java index 81c3b5f..840f16a 100644 --- a/clients/src/main/java/org/apache/kafka/clients/consumer/KafkaConsumer.java +++ b/clients/src/main/java/org/apache/kafka/clients/consumer/KafkaConsumer.java @@ -650,7 +650,7 @@ public class KafkaConsumer implements Consumer { // fetch positions if we have partitions we're subscribed to that we // don't know the offset for if (!subscriptions.hasAllFetchPositions()) - fetcher.updateFetchPositions(this.subscriptions.missingFetchPositions(), now); + updateFetchPositions(this.subscriptions.missingFetchPositions(), now); // maybe autocommit position if (shouldAutoCommit(now)) @@ -695,7 +695,10 @@ public class KafkaConsumer implements Consumer { this.lastCommitAttemptMs = now; // commit the offsets with the coordinator - coordinator.commitOffsets(offsets, commitType.equals(CommitType.SYNC), now); + boolean syncCommit = commitType.equals(CommitType.SYNC); + if (!syncCommit) + this.subscriptions.commitInProgress(); + coordinator.commitOffsets(offsets, syncCommit, now); } /** @@ -765,7 +768,7 @@ public class KafkaConsumer implements Consumer { throw new IllegalArgumentException("You can only check the position for partitions assigned to this consumer."); Long offset = this.subscriptions.consumed(partition); if (offset == null) { - fetcher.updateFetchPositions(Collections.singleton(partition), time.milliseconds()); + updateFetchPositions(Collections.singleton(partition), time.milliseconds()); return this.subscriptions.consumed(partition); } else { return offset; @@ -855,20 +858,48 @@ public class KafkaConsumer implements Consumer { } /** - * Refresh the committed offsets for given set of partitions and update the cache + * Set the fetch position to the committed position (if there is one) + * or reset it using the offset reset policy the user has configured. * - * @param partitions The list of partitions that needs refreshing committed offsets + * @param partitions The partitions that needs updating fetch positions * @param now The current time + * @throws org.apache.kafka.clients.consumer.NoOffsetForPartitionException If no offset is stored for a given partition and no offset reset policy is + * defined */ - private void refreshCommittedOffsets(Set partitions, long now) { + private void updateFetchPositions(Set partitions, long now) { + // first refresh the committed positions in case they are not up-to-date + refreshCommittedOffsets(partitions, now); - // contact coordinator to fetch committed offsets - Map offsets = coordinator.fetchOffsets(partitions, now); + // reset the fetch position to the committed position + for (TopicPartition tp : partitions) { + if (subscriptions.fetched(tp) == null) { + if (subscriptions.committed(tp) == null) { + // if the committed position is unknown reset the position + fetcher.resetOffset(tp); + } else { + log.debug("Resetting offset for partition {} to committed offset"); + subscriptions.seek(tp, subscriptions.committed(tp)); + } + } + } + } - // update the position with the offsets - for (Map.Entry entry : offsets.entrySet()) { - TopicPartition tp = entry.getKey(); - this.subscriptions.committed(tp, entry.getValue()); + /** + * Refresh the committed offsets for given set of partitions and update the cache + */ + private void refreshCommittedOffsets(Set partitions, long now) { + // we only need to fetch latest committed offset from coordinator if there + // is some commit process in progress, otherwise our current + // committed cache is up-to-date + if (subscriptions.needsSyncCommits()) { + // contact coordinator to fetch committed offsets + Map offsets = coordinator.fetchOffsets(partitions, now); + + // update the position with the offsets + for (Map.Entry entry : offsets.entrySet()) { + TopicPartition tp = entry.getKey(); + this.subscriptions.committed(tp, entry.getValue()); + } } } diff --git a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/FetchManager.java b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/FetchManager.java index 5beb0c2..70905ab 100644 --- a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/FetchManager.java +++ b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/FetchManager.java @@ -152,26 +152,23 @@ public class FetchManager { } /** - * Set the fetch position to the committed position (if there is one) - * or reset it using the offset reset policy the user has configured. + * Reset offsets for the given partition using the offset reset strategy. * - * @param partitions The partitions that needs updating fetch positions - * @param now The current time - * @throws org.apache.kafka.clients.consumer.NoOffsetForPartitionException If no offset is stored for a given partition and no offset reset policy is - * defined + * @param partition The given partition that needs reset offset + * @throws org.apache.kafka.clients.consumer.NoOffsetForPartitionException If no offset reset strategy is defined */ - public void updateFetchPositions(Set partitions, long now) { - // reset the fetch position to the committed position - for (TopicPartition tp : partitions) { - if (subscriptions.fetched(tp) == null) { - if (subscriptions.committed(tp) == null) { - resetOffset(tp, now); - } else { - log.debug("Resetting offset for partition {} to committed offset"); - subscriptions.seek(tp, subscriptions.committed(tp)); - } - } - } + public void resetOffset(TopicPartition partition) { + long timestamp; + if (this.offsetResetStrategy == AutoOffsetResetStrategy.EARLIEST) + timestamp = EARLIEST_OFFSET_TIMESTAMP; + else if (this.offsetResetStrategy == AutoOffsetResetStrategy.LATEST) + timestamp = LATEST_OFFSET_TIMESTAMP; + else + throw new NoOffsetForPartitionException("No offset is set and no reset policy is defined"); + + log.debug("Resetting offset for partition {} to {} offset.", partition, this.offsetResetStrategy.name() + .toLowerCase()); + this.subscriptions.seek(partition, offsetBefore(partition, timestamp)); } /** @@ -310,7 +307,7 @@ public class FetchManager { this.metadata.requestUpdate(); } else if (partition.errorCode == Errors.OFFSET_OUT_OF_RANGE.code()) { // TODO: this could be optimized by grouping all out-of-range partitions - resetOffset(tp, time.milliseconds()); + resetOffset(tp); } } this.metrics.bytesFetched.record(totalBytes); @@ -346,25 +343,6 @@ public class FetchManager { } while (this.metadata.version() == version); } - /** - * Reset offsets for the given partition using the offset reset strategy - * - * @throws org.apache.kafka.clients.consumer.NoOffsetForPartitionException If no offset reset strategy is defined - */ - private void resetOffset(TopicPartition partition, long now) { - long timestamp; - if (this.offsetResetStrategy == AutoOffsetResetStrategy.EARLIEST) - timestamp = EARLIEST_OFFSET_TIMESTAMP; - else if (this.offsetResetStrategy == AutoOffsetResetStrategy.LATEST) - timestamp = LATEST_OFFSET_TIMESTAMP; - else - throw new NoOffsetForPartitionException("No offset is set and no reset policy is defined"); - - log.debug("Resetting offset for partition {} to {} offset.", partition, this.offsetResetStrategy.name() - .toLowerCase()); - this.subscriptions.seek(partition, offsetBefore(partition, timestamp)); - } - private static class PartitionRecords { public long fetchOffset; public TopicPartition partition; diff --git a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/SubscriptionState.java b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/SubscriptionState.java index d41d306..5e4e9ba 100644 --- a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/SubscriptionState.java +++ b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/SubscriptionState.java @@ -43,9 +43,12 @@ public class SubscriptionState { /* the last committed offset for each partition */ private final Map committed; - /* do we need to request a partition assignment from the co-ordinator? */ + /* do we need to request a partition assignment from the coordinator? */ private boolean needsPartitionAssignment; + /* is there an asynchronous commit request on-going with the coordinator? */ + private boolean commitInProgress; + public SubscriptionState() { this.subscribedTopics = new HashSet(); this.subscribedPartitions = new HashSet(); @@ -54,6 +57,7 @@ public class SubscriptionState { this.fetched = new HashMap(); this.committed = new HashMap(); this.needsPartitionAssignment = false; + this.commitInProgress = true; // initialize to true for the consumers to fetch offset upon starting up } public void subscribe(String topic) { @@ -119,11 +123,20 @@ public class SubscriptionState { public void committed(TopicPartition tp, long offset) { this.committed.put(tp, offset); + this.commitInProgress = false; } public Long committed(TopicPartition tp) { return this.committed.get(tp); } + + public void commitInProgress() { + this.commitInProgress = true; + } + + public boolean needsSyncCommits() { + return this.commitInProgress; + } public void seek(TopicPartition tp, long offset) { fetched(tp, offset); -- 1.7.12.4 From 30cf1637016033c755a6abdc99202053b4ccbacf Mon Sep 17 00:00:00 2001 From: Guozhang Wang Date: Thu, 19 Feb 2015 18:05:41 -0800 Subject: [PATCH 06/11] test dummy --- .../org/apache/kafka/clients/consumer/internals/Coordinator.java | 3 ++- core/src/test/resources/log4j.properties | 6 +++--- core/src/test/scala/integration/kafka/api/ConsumerTest.scala | 6 +++--- 3 files changed, 8 insertions(+), 7 deletions(-) diff --git a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/Coordinator.java b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/Coordinator.java index cd7ae04..e95e6db 100644 --- a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/Coordinator.java +++ b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/Coordinator.java @@ -395,7 +395,8 @@ public final class Coordinator { response.request().request().destination()); // mark the coordinator as dead - coordinatorDead(); + if (this.consumerCoordinator != null) + coordinatorDead(); } /** diff --git a/core/src/test/resources/log4j.properties b/core/src/test/resources/log4j.properties index 1b7d5d8..e442b72 100644 --- a/core/src/test/resources/log4j.properties +++ b/core/src/test/resources/log4j.properties @@ -12,14 +12,14 @@ # 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. -log4j.rootLogger=OFF, stdout +log4j.rootLogger=DEBUG, stdout log4j.appender.stdout=org.apache.log4j.ConsoleAppender log4j.appender.stdout.layout=org.apache.log4j.PatternLayout log4j.appender.stdout.layout.ConversionPattern=[%d] %p %m (%c:%L)%n -log4j.logger.kafka=ERROR -log4j.logger.org.apache.kafka=ERROR +log4j.logger.kafka=INFO +log4j.logger.org.apache.kafka=DEBUG # zkclient can be verbose, during debugging it is common to adjust is separately log4j.logger.org.I0Itec.zkclient.ZkClient=WARN diff --git a/core/src/test/scala/integration/kafka/api/ConsumerTest.scala b/core/src/test/scala/integration/kafka/api/ConsumerTest.scala index 85c6770..b77f157 100644 --- a/core/src/test/scala/integration/kafka/api/ConsumerTest.scala +++ b/core/src/test/scala/integration/kafka/api/ConsumerTest.scala @@ -214,6 +214,9 @@ class ConsumerTest extends IntegrationTestHarness with Logging { } def testPartitionReassignmentCallback() { + // create the topic with replication + TestUtils.createTopic(this.zkClient, topic, 1, serverCount, this.servers) + val callback = new TestConsumerReassignmentCallback() this.consumerConfig.setProperty(ConsumerConfig.SESSION_TIMEOUT_MS_CONFIG, "200"); // timeout quickly to avoid slow test val consumer0 = new KafkaConsumer(this.consumerConfig, callback, new ByteArrayDeserializer(), new ByteArrayDeserializer()) @@ -240,9 +243,6 @@ class ConsumerTest extends IntegrationTestHarness with Logging { assertEquals(2, callback.callsToAssigned) assertEquals(2, callback.callsToRevoked) - // restart the coordinator since it may also be hosting "test" topic - this.servers(coordinator).startup() - consumer0.close() } -- 1.7.12.4 From 5d4c99dd7e9b7433e0a5259ce8b28f158f5df4eb Mon Sep 17 00:00:00 2001 From: Guozhang Wang Date: Mon, 23 Feb 2015 17:57:58 -0800 Subject: [PATCH 07/11] review for Onur --- .../kafka/clients/consumer/KafkaConsumer.java | 1 + .../clients/consumer/internals/Coordinator.java | 24 +++++++++++--- .../consumer/internals/SubscriptionState.java | 4 +++ .../org/apache/kafka/common/network/Selector.java | 3 +- .../org/apache/kafka/common/protocol/Errors.java | 24 +++++++++----- .../kafka/common/requests/JoinGroupRequest.java | 2 ++ .../kafka/coordinator/ConsumerCoordinator.scala | 38 +++++++++++++++------- .../scala/kafka/coordinator/ConsumerRegistry.scala | 8 ++--- .../scala/kafka/coordinator/DelayedHeartbeat.scala | 2 +- .../scala/kafka/coordinator/GroupRegistry.scala | 7 +++- core/src/main/scala/kafka/server/KafkaServer.scala | 2 +- core/src/test/resources/log4j.properties | 6 ++-- .../scala/integration/kafka/api/ConsumerTest.scala | 14 ++++++-- .../kafka/api/IntegrationTestHarness.scala | 7 ++++ 14 files changed, 103 insertions(+), 39 deletions(-) diff --git a/clients/src/main/java/org/apache/kafka/clients/consumer/KafkaConsumer.java b/clients/src/main/java/org/apache/kafka/clients/consumer/KafkaConsumer.java index 840f16a..3d1c9e1 100644 --- a/clients/src/main/java/org/apache/kafka/clients/consumer/KafkaConsumer.java +++ b/clients/src/main/java/org/apache/kafka/clients/consumer/KafkaConsumer.java @@ -491,6 +491,7 @@ public class KafkaConsumer implements Consumer { this.retryBackoffMs, config.getLong(ConsumerConfig.SESSION_TIMEOUT_MS_CONFIG), config.getString(ConsumerConfig.PARTITION_ASSIGNMENT_STRATEGY_CONFIG), + this.metadata, this.subscriptions, this.metrics, this.time); diff --git a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/Coordinator.java b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/Coordinator.java index e95e6db..3409a3a 100644 --- a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/Coordinator.java +++ b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/Coordinator.java @@ -15,6 +15,7 @@ package org.apache.kafka.clients.consumer.internals; import org.apache.kafka.clients.ClientRequest; import org.apache.kafka.clients.ClientResponse; import org.apache.kafka.clients.KafkaClient; +import org.apache.kafka.clients.Metadata; import org.apache.kafka.clients.RequestCompletionHandler; import org.apache.kafka.clients.consumer.KafkaConsumer.ConsumerMetrics; import org.apache.kafka.common.KafkaException; @@ -58,6 +59,7 @@ public final class Coordinator { private final Time time; private final String groupId; + private final Metadata metadata; private final Heartbeat heartbeat; private final long sessionTimeoutMs; private final String assignmentStrategy; @@ -71,7 +73,7 @@ public final class Coordinator { @Override public void onComplete(ClientResponse resp) { if (resp.wasDisconnected()) { - coordinatorDead(); + handleCoordinatorDisconnect(resp); } else { HeartbeatResponse response = new HeartbeatResponse(resp.responseBody()); if (response.errorCode() == Errors.NONE.code()) { @@ -80,7 +82,10 @@ public final class Coordinator { } else if (response.errorCode() == Errors.CONSUMER_COORDINATOR_NOT_AVAILABLE.code() || response.errorCode() == Errors.NOT_COORDINATOR_FOR_CONSUMER.code()) { coordinatorDead(); - } else { + } else if (response.errorCode() == Errors.ILLEGAL_CONSUMER_GENERATION.code()) { + subscriptions.needReassignment(); + } + else { throw new KafkaException("Unexpected error in heartbeat response: " + Errors.forCode(response.errorCode()).exception().getMessage()); } @@ -133,6 +138,7 @@ public final class Coordinator { long retryBackoffMs, long sessionTimeoutMs, String assignmentStrategy, + Metadata metadata, SubscriptionState subscriptions, ConsumerMetrics metrics, Time time) { @@ -143,6 +149,7 @@ public final class Coordinator { this.generation = -1; this.consumerId = ""; this.groupId = groupId; + this.metadata = metadata; this.consumerCoordinator = null; this.subscriptions = subscriptions; this.retryBackoffMs = retryBackoffMs; @@ -343,14 +350,14 @@ public final class Coordinator { */ private void discoverCoordinator() { while (this.consumerCoordinator == null) { - log.debug("No consumer coordinator known, attempting to discover one."); + log.debug("No coordinator known, attempting to discover one."); Node coordinator = fetchConsumerCoordinator(); if (coordinator == null) { log.debug("No coordinator found, backing off."); Utils.sleep(this.retryBackoffMs); } else { - log.debug("Found consumer coordinator: " + coordinator); + log.debug("Found coordinator: " + coordinator); this.consumerCoordinator = coordinator; } } @@ -369,7 +376,8 @@ public final class Coordinator { // send the request and wait for its response ClientResponse response = sendAndReceive(request, request.createdTime()); - // parse the response to get the coordinator info if it is not disconnected + // parse the response to get the coordinator info if it is not disconnected, + // otherwise we need to request metadata update if (!response.wasDisconnected()) { ConsumerMetadataResponse consumerMetadataResponse = new ConsumerMetadataResponse(response.responseBody()); // use MAX_VALUE - node.id as the coordinator id to mimic separate connections @@ -379,6 +387,8 @@ public final class Coordinator { return new Node(Integer.MAX_VALUE - consumerMetadataResponse.node().id(), consumerMetadataResponse.node().host(), consumerMetadataResponse.node().port()); + } else { + this.metadata.requestUpdate(); } return null; @@ -412,6 +422,10 @@ public final class Coordinator { long now = time.milliseconds(); this.client.poll(this.retryBackoffMs, now); node = this.client.leastLoadedNode(now); + + // if there is no ready node, backoff before retry + if (node == null) + Utils.sleep(this.retryBackoffMs); } // create a consumer metadata request diff --git a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/SubscriptionState.java b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/SubscriptionState.java index 5e4e9ba..d9df339 100644 --- a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/SubscriptionState.java +++ b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/SubscriptionState.java @@ -79,6 +79,10 @@ public class SubscriptionState { clearPartition(tp); } + public void needReassignment() { + this.needsPartitionAssignment = true; + } + public void subscribe(TopicPartition tp) { if (this.subscribedTopics.size() > 0) throw new IllegalStateException("Subcription to topics and partitions are mutually exclusive"); 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 6baad93..db9b9a6 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 @@ -14,6 +14,7 @@ package org.apache.kafka.common.network; import java.io.EOFException; import java.io.IOException; +import java.net.ConnectException; import java.net.InetSocketAddress; import java.net.Socket; import java.nio.channels.CancelledKeyException; @@ -274,7 +275,7 @@ public class Selector implements Selectable { } } catch (IOException e) { String desc = socketDescription(channel); - if (e instanceof EOFException) + if (e instanceof EOFException || e instanceof ConnectException) log.info("Connection {} disconnected", desc); else log.warn("Error in I/O with connection to {}", desc, e); diff --git a/clients/src/main/java/org/apache/kafka/common/protocol/Errors.java b/clients/src/main/java/org/apache/kafka/common/protocol/Errors.java index ad2171f..2ec86b2 100644 --- a/clients/src/main/java/org/apache/kafka/common/protocol/Errors.java +++ b/clients/src/main/java/org/apache/kafka/common/protocol/Errors.java @@ -41,14 +41,20 @@ public enum Errors { new LeaderNotAvailableException("There is no leader for this topic-partition as we are in the middle of a leadership election.")), NOT_LEADER_FOR_PARTITION(6, new NotLeaderForPartitionException("This server is not the leader for that topic-partition.")), - REQUEST_TIMED_OUT(7, new TimeoutException("The request timed out.")), + REQUEST_TIMED_OUT(7, + new TimeoutException("The request timed out.")), MESSAGE_TOO_LARGE(10, new RecordTooLargeException("The request included a message larger than the max message size the server will accept.")), - OFFSET_METADATA_TOO_LARGE(12, new OffsetMetadataTooLarge("The metadata field of the offset request was too large.")), - NETWORK_EXCEPTION(13, new NetworkException("The server disconnected before a response was received.")), - OFFSET_LOAD_IN_PROGRESS(14, new ApiException("The coordinator is loading offsets and can't process requests.")), - CONSUMER_COORDINATOR_NOT_AVAILABLE(15, new ApiException("The coordinator is not available.")), - NOT_COORDINATOR_FOR_CONSUMER(16, new ApiException("This is not the correct co-ordinator for this consumer.")), + OFFSET_METADATA_TOO_LARGE(12, + new OffsetMetadataTooLarge("The metadata field of the offset request was too large.")), + NETWORK_EXCEPTION(13, + new NetworkException("The server disconnected before a response was received.")), + OFFSET_LOAD_IN_PROGRESS(14, + new ApiException("The coordinator is loading offsets and can't process requests.")), + CONSUMER_COORDINATOR_NOT_AVAILABLE(15, + new ApiException("The coordinator is not available.")), + NOT_COORDINATOR_FOR_CONSUMER(16, + new ApiException("This is not the correct co-ordinator for this consumer.")), INVALID_TOPIC_EXCEPTION(17, new InvalidTopicException("The request attempted to perform an operation on an invalid topic.")), RECORD_LIST_TOO_LARGE(18, @@ -57,7 +63,10 @@ public enum Errors { new NotEnoughReplicasException("Messages are rejected since there are fewer in-sync replicas than required.")), NOT_ENOUGH_REPLICAS_AFTER_APPEND(20, new NotEnoughReplicasAfterAppendException("Messages are written to the log, but to fewer in-sync replicas than required.")), - INVALID_REQUIRED_ACKS(21, new InvalidRequiredAcksException("Produce request specified an invalid value for required acks.")); + INVALID_REQUIRED_ACKS(21, + new InvalidRequiredAcksException("Produce request specified an invalid value for required acks.")), + ILLEGAL_CONSUMER_GENERATION(22, + new ApiException("Specified consumer generation id is not valid.")); private static Map, Errors> classToError = new HashMap, Errors>(); private static Map codeToError = new HashMap(); @@ -68,7 +77,6 @@ public enum Errors { if (error.exception != null) classToError.put(error.exception.getClass(), error); } - } private final short code; diff --git a/clients/src/main/java/org/apache/kafka/common/requests/JoinGroupRequest.java b/clients/src/main/java/org/apache/kafka/common/requests/JoinGroupRequest.java index 8c50e9b..1ebc188 100644 --- a/clients/src/main/java/org/apache/kafka/common/requests/JoinGroupRequest.java +++ b/clients/src/main/java/org/apache/kafka/common/requests/JoinGroupRequest.java @@ -30,6 +30,8 @@ public class JoinGroupRequest extends AbstractRequestResponse { private static final String CONSUMER_ID_KEY_NAME = "consumer_id"; private static final String STRATEGY_KEY_NAME = "partition_assignment_strategy"; + public static final String UNKNOWN_CONSUMER_ID = ""; + private final String groupId; private final int sessionTimeout; private final List topics; diff --git a/core/src/main/scala/kafka/coordinator/ConsumerCoordinator.scala b/core/src/main/scala/kafka/coordinator/ConsumerCoordinator.scala index 21790a5..2b63b37 100644 --- a/core/src/main/scala/kafka/coordinator/ConsumerCoordinator.scala +++ b/core/src/main/scala/kafka/coordinator/ConsumerCoordinator.scala @@ -25,6 +25,7 @@ import kafka.utils._ import scala.collection.mutable.HashMap import org.I0Itec.zkclient.{IZkChildListener, ZkClient} +import org.apache.kafka.common.requests.JoinGroupRequest /** @@ -114,9 +115,15 @@ class ConsumerCoordinator(val config: KafkaConfig, if (!consumerGroupRegistries.contains(groupId)) createNewGroup(groupId, partitionAssignmentStrategy) + val groupRegistry = consumerGroupRegistries(groupId) + // if the consumer id is unknown or it does exists in // the group yet, register this consumer to the group - // TODO + if (consumerId.equals(JoinGroupRequest.UNKNOWN_CONSUMER_ID)) { + createNewConsumer(groupId, groupRegistry.generateNextConsumerId, topics, sessionTimeoutMs) + } else if (!groupRegistry.memberRegistries.contains(consumerId)) { + createNewConsumer(groupId, consumerId, topics, sessionTimeoutMs) + } // add a delayed join-group operation to the purgatory // TODO @@ -146,9 +153,9 @@ class ConsumerCoordinator(val config: KafkaConfig, * Process a heartbeat request from a consumer */ def consumerHeartbeat(groupId: String, - consumerId: String, - generationId: Int, - responseCallback: Short => Unit) { + consumerId: String, + generationId: Int, + responseCallback: Short => Unit) { // check that the group already exists // TODO @@ -171,21 +178,28 @@ class ConsumerCoordinator(val config: KafkaConfig, // TODO: this is just a stub for new consumer testing, // TODO: needs to be replaced with the logic above // TODO -------------------------------------------------------------- - // always return OK for heartbeat immediately - responseCallback(Errors.NONE.code) + // check if the consumer already exist, if yes return OK, + // otherwise return illegal generation error + if (consumerGroupRegistries.contains(groupId) + && consumerGroupRegistries(groupId).memberRegistries.contains(consumerId)) + responseCallback(Errors.NONE.code) + else + responseCallback(Errors.ILLEGAL_CONSUMER_GENERATION.code) } /** * Create a new consumer */ - private def createNewConsumer(consumerId: String, + private def createNewConsumer(groupId: String, + consumerId: String, topics: List[String], - sessionTimeoutMs: Int, - groupRegistry: GroupRegistry) { - debug("Registering consumer " + consumerId + " for group " + groupRegistry.groupId) + sessionTimeoutMs: Int) { + debug("Registering consumer " + consumerId + " for group " + groupId) // create the new consumer registry entry - // TODO: specify consumerId as unknown and update at the end of the prepare-rebalance phase + val consumerRegistry = new ConsumerRegistry(groupId, consumerId, topics, sessionTimeoutMs) + + consumerGroupRegistries(groupId).memberRegistries.put(consumerId, consumerRegistry) // check if the partition assignment strategy is consistent with the group // TODO @@ -202,7 +216,7 @@ class ConsumerCoordinator(val config: KafkaConfig, // start preparing group partition rebalance // TODO - info("Registered consumer " + consumerId + " for group " + groupRegistry.groupId) + info("Registered consumer " + consumerId + " for group " + groupId) } /** diff --git a/core/src/main/scala/kafka/coordinator/ConsumerRegistry.scala b/core/src/main/scala/kafka/coordinator/ConsumerRegistry.scala index b65c04d..2f57970 100644 --- a/core/src/main/scala/kafka/coordinator/ConsumerRegistry.scala +++ b/core/src/main/scala/kafka/coordinator/ConsumerRegistry.scala @@ -32,10 +32,10 @@ import java.util.HashMap * 1. subscribed topic list * 2. assigned partitions for the subscribed topics. */ -class ConsumerRegistry(val consumerId: String, - val subscribedTopics: List[String], - val sessionTimeoutMs: Int, - val groupRegistry: GroupRegistry) { +class ConsumerRegistry(val groupId: String, + val consumerId: String, + val topics: List[String], + val sessionTimeoutMs: Int) { /* number of expired heartbeat recorded */ val numExpiredHeartbeat = new AtomicInteger(0) diff --git a/core/src/main/scala/kafka/coordinator/DelayedHeartbeat.scala b/core/src/main/scala/kafka/coordinator/DelayedHeartbeat.scala index b1248e9..6a6bc7b 100644 --- a/core/src/main/scala/kafka/coordinator/DelayedHeartbeat.scala +++ b/core/src/main/scala/kafka/coordinator/DelayedHeartbeat.scala @@ -43,6 +43,6 @@ class DelayedHeartbeat(sessionTimeout: Long, /* mark all consumers within the heartbeat as heartbeat timed out */ override def onComplete() { for (registry <- bucket.consumerRegistryList) - expireCallback(registry.groupRegistry.groupId, registry.consumerId) + expireCallback(registry.groupId, registry.consumerId) } } diff --git a/core/src/main/scala/kafka/coordinator/GroupRegistry.scala b/core/src/main/scala/kafka/coordinator/GroupRegistry.scala index 7d17e10..cd3864a 100644 --- a/core/src/main/scala/kafka/coordinator/GroupRegistry.scala +++ b/core/src/main/scala/kafka/coordinator/GroupRegistry.scala @@ -18,6 +18,7 @@ package kafka.coordinator import scala.collection.mutable +import java.util.concurrent.atomic.AtomicInteger sealed trait GroupStates { def state: Byte } @@ -69,6 +70,10 @@ class GroupRegistry(val groupId: String, val state: GroupState = new GroupState() - var generationId: Int = 1 + var generationId = new AtomicInteger(1) + + var nextConsumerId = new AtomicInteger(1) + + def generateNextConsumerId = groupId + "-" + nextConsumerId.getAndIncrement.toString } diff --git a/core/src/main/scala/kafka/server/KafkaServer.scala b/core/src/main/scala/kafka/server/KafkaServer.scala index 426e522..4f30901 100644 --- a/core/src/main/scala/kafka/server/KafkaServer.scala +++ b/core/src/main/scala/kafka/server/KafkaServer.scala @@ -307,7 +307,7 @@ class KafkaServer(val config: KafkaConfig, time: Time = SystemTime) extends Logg throw new IllegalStateException("Kafka server is still starting up, cannot shut down!") val canShutdown = isShuttingDown.compareAndSet(false, true) - if (canShutdown) { + if (canShutdown && shutdownLatch.getCount > 0) { Utils.swallow(controlledShutdown()) brokerState.newState(BrokerShuttingDown) if(socketServer != null) diff --git a/core/src/test/resources/log4j.properties b/core/src/test/resources/log4j.properties index e442b72..9973dad 100644 --- a/core/src/test/resources/log4j.properties +++ b/core/src/test/resources/log4j.properties @@ -12,14 +12,14 @@ # 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. -log4j.rootLogger=DEBUG, stdout +log4j.rootLogger=TRACE, stdout log4j.appender.stdout=org.apache.log4j.ConsoleAppender log4j.appender.stdout.layout=org.apache.log4j.PatternLayout log4j.appender.stdout.layout.ConversionPattern=[%d] %p %m (%c:%L)%n -log4j.logger.kafka=INFO -log4j.logger.org.apache.kafka=DEBUG +log4j.logger.kafka=TRACE +log4j.logger.org.apache.kafka=TRACE # zkclient can be verbose, during debugging it is common to adjust is separately log4j.logger.org.I0Itec.zkclient.ZkClient=WARN diff --git a/core/src/test/scala/integration/kafka/api/ConsumerTest.scala b/core/src/test/scala/integration/kafka/api/ConsumerTest.scala index b77f157..db4aa6d 100644 --- a/core/src/test/scala/integration/kafka/api/ConsumerTest.scala +++ b/core/src/test/scala/integration/kafka/api/ConsumerTest.scala @@ -62,6 +62,8 @@ class ConsumerTest extends IntegrationTestHarness with Logging { } def testSimpleConsumption() { + TestUtils.createTopic(this.zkClient, topic, 1, serverCount, this.servers) + val numRecords = 10000 sendRecords(numRecords) @@ -74,12 +76,16 @@ class ConsumerTest extends IntegrationTestHarness with Logging { } def testAutoOffsetReset() { + TestUtils.createTopic(this.zkClient, topic, 1, serverCount, this.servers) + sendRecords(1) this.consumers(0).subscribe(tp) consumeRecords(this.consumers(0), numRecords = 1, startingOffset = 0) } def testSeek() { + TestUtils.createTopic(this.zkClient, topic, 1, serverCount, this.servers) + val consumer = this.consumers(0) val totalRecords = 50L sendRecords(totalRecords.toInt) @@ -100,7 +106,8 @@ class ConsumerTest extends IntegrationTestHarness with Logging { } def testGroupConsumption() { - // we need to do this test with only one server since we have the hack join group + TestUtils.createTopic(this.zkClient, topic, 1, serverCount, this.servers) + // we need to do this test with only one server since we have the hack join group // that just assigns the partition hosted on the local machine (with two we might get the wrong machine this.servers.last.shutdown() this.servers.head.shutdown() @@ -110,6 +117,7 @@ class ConsumerTest extends IntegrationTestHarness with Logging { } def testPositionAndCommit() { + TestUtils.createTopic(this.zkClient, topic, 1, serverCount, this.servers) sendRecords(5) // committed() on a partition with no committed offset throws an exception @@ -168,7 +176,7 @@ class ConsumerTest extends IntegrationTestHarness with Logging { assertEquals(consumed.toLong, record.offset()) consumed += 1 } - consumer.commit(CommitType.SYNC); + consumer.commit(CommitType.SYNC) /* restart any dead brokers, and kill a broker (with probability 1/3) */ restartDeadBrokers() @@ -220,7 +228,7 @@ class ConsumerTest extends IntegrationTestHarness with Logging { val callback = new TestConsumerReassignmentCallback() this.consumerConfig.setProperty(ConsumerConfig.SESSION_TIMEOUT_MS_CONFIG, "200"); // timeout quickly to avoid slow test val consumer0 = new KafkaConsumer(this.consumerConfig, callback, new ByteArrayDeserializer(), new ByteArrayDeserializer()) - consumer0.subscribe("test") + consumer0.subscribe(topic) // the initial subscription should cause a callback execution while(callback.callsToAssigned == 0) diff --git a/core/src/test/scala/integration/kafka/api/IntegrationTestHarness.scala b/core/src/test/scala/integration/kafka/api/IntegrationTestHarness.scala index 5650b4a..c58b7a9 100644 --- a/core/src/test/scala/integration/kafka/api/IntegrationTestHarness.scala +++ b/core/src/test/scala/integration/kafka/api/IntegrationTestHarness.scala @@ -62,6 +62,13 @@ trait IntegrationTestHarness extends KafkaServerTestHarness { producers += new KafkaProducer(producerConfig) for(i <- 0 until consumerCount) consumers += new KafkaConsumer(consumerConfig) + + // make sure the consumer's metadata has been refreshed with the full broker list + for(i <- 0 until consumerCount) { + while consumers(i) + } + + } override def tearDown() { -- 1.7.12.4 From a3226e92e61e9b7dc32f0bce6139a424ce923be3 Mon Sep 17 00:00:00 2001 From: Guozhang Wang Date: Tue, 24 Feb 2015 16:34:31 -0800 Subject: [PATCH 08/11] fix tests --- .../clients/consumer/internals/Coordinator.java | 74 ++++++++++++++-------- .../clients/consumer/internals/FetchManager.java | 7 +- .../org/apache/kafka/common/protocol/Errors.java | 7 +- .../kafka/common/requests/FetchResponse.java | 11 ++++ .../kafka/common/requests/HeartbeatResponse.java | 6 ++ .../kafka/common/requests/JoinGroupResponse.java | 7 ++ .../kafka/common/requests/ListOffsetResponse.java | 7 ++ .../kafka/common/requests/MetadataResponse.java | 14 ++++ .../common/requests/OffsetCommitResponse.java | 6 ++ .../kafka/common/requests/OffsetFetchResponse.java | 9 +++ .../kafka/common/requests/ProduceResponse.java | 7 ++ .../src/main/scala/kafka/common/ErrorMapping.scala | 6 +- .../kafka/common/NoOffsetsCommittedException.scala | 27 ++++++++ .../kafka/common/OffsetMetadataAndError.scala | 2 +- core/src/test/resources/log4j.properties | 6 +- .../scala/integration/kafka/api/ConsumerTest.scala | 2 +- .../kafka/api/IntegrationTestHarness.scala | 17 ++--- 17 files changed, 170 insertions(+), 45 deletions(-) create mode 100644 core/src/main/scala/kafka/common/NoOffsetsCommittedException.scala diff --git a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/Coordinator.java b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/Coordinator.java index 3409a3a..537c266 100644 --- a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/Coordinator.java +++ b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/Coordinator.java @@ -84,8 +84,7 @@ public final class Coordinator { coordinatorDead(); } else if (response.errorCode() == Errors.ILLEGAL_CONSUMER_GENERATION.code()) { subscriptions.needReassignment(); - } - else { + } else { throw new KafkaException("Unexpected error in heartbeat response: " + Errors.forCode(response.errorCode()).exception().getMessage()); } @@ -246,31 +245,52 @@ public final class Coordinator { public Map fetchOffsets(Set partitions, long now) { log.debug("Fetching committed offsets for partitions: " + Utils.join(partitions, ", ")); - // construct the request - OffsetFetchRequest request = new OffsetFetchRequest(this.groupId, new ArrayList(partitions)); - - // send the request and block on waiting for response - ClientResponse resp = this.blockingCoordinatorRequest(ApiKeys.OFFSET_FETCH, request.toStruct(), null, now); - - // parse the response to get the offsets - OffsetFetchResponse response = new OffsetFetchResponse(resp.responseBody()); - Map offsets = new HashMap(response.responseData().size()); - for (Map.Entry entry : response.responseData().entrySet()) { - TopicPartition tp = entry.getKey(); - OffsetFetchResponse.PartitionData data = entry.getValue(); - if (data.hasError()) { - log.debug("Error fetching offset for topic-partition {}: {}", tp, Errors.forCode(data.errorCode) - .exception() - .getMessage()); - } else if (data.offset >= 0) { - // record the position with the offset (-1 seems to indicate no - // such offset known) - offsets.put(tp, data.offset); - } else { - log.debug("No committed offset for partition " + tp); + while(true) { + // construct the request + OffsetFetchRequest request = new OffsetFetchRequest(this.groupId, new ArrayList(partitions)); + + // send the request and block on waiting for response + ClientResponse resp = this.blockingCoordinatorRequest(ApiKeys.OFFSET_FETCH, request.toStruct(), null, now); + + // parse the response to get the offsets + boolean offsetsReady = true; + OffsetFetchResponse response = new OffsetFetchResponse(resp.responseBody()); + Map offsets = new HashMap(response.responseData().size()); + for (Map.Entry entry : response.responseData().entrySet()) { + TopicPartition tp = entry.getKey(); + OffsetFetchResponse.PartitionData data = entry.getValue(); + if (data.hasError()) { + log.debug("Error fetching offset for topic-partition {}: {}", tp, Errors.forCode(data.errorCode) + .exception() + .getMessage()); + if (data.errorCode == Errors.OFFSET_LOAD_IN_PROGRESS.code()) { + // just retry + offsetsReady = false; + Utils.sleep(this.retryBackoffMs); + } else if (data.errorCode == Errors.NOT_COORDINATOR_FOR_CONSUMER.code()) { + // re-discover the coordinator and retry + coordinatorDead(); + offsetsReady = false; + Utils.sleep(this.retryBackoffMs); + } else if (data.errorCode == Errors.NO_OFFSETS_COMMITTED.code() + || data.errorCode == Errors.UNKNOWN_TOPIC_OR_PARTITION.code()) { + // just ignore this partition + log.debug("No committed offset for partition " + tp); + } else { + throw new IllegalStateException("Unexpected error code " + data.errorCode + " while fetching offset"); + } + } else if (data.offset >= 0) { + // record the position with the offset (-1 seems to indicate no + // such offset known) + offsets.put(tp, data.offset); + } else { + log.debug("No committed offset for partition " + tp); + } } + + if (offsetsReady) + return offsets; } - return offsets; } /** @@ -326,7 +346,9 @@ public final class Coordinator { // if the coordinator connection has failed, we need to // break the inner loop to re-discover the coordinator - if (this.client.connectionFailed(this.consumerCoordinator)) { + if (this.consumerCoordinator == null) { + break; + } else if (this.client.connectionFailed(this.consumerCoordinator)) { log.debug("Coordinator connection failed. Attempting to re-discover."); coordinatorDead(); break; diff --git a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/FetchManager.java b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/FetchManager.java index 70905ab..346ce00 100644 --- a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/FetchManager.java +++ b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/FetchManager.java @@ -302,12 +302,15 @@ public class FetchManager { totalBytes += bytes; totalCount += parsed.size(); } else if (partition.errorCode == Errors.NOT_LEADER_FOR_PARTITION.code() - || partition.errorCode == Errors.UNKNOWN_TOPIC_OR_PARTITION.code() - || partition.errorCode == Errors.LEADER_NOT_AVAILABLE.code()) { + || partition.errorCode == Errors.UNKNOWN_TOPIC_OR_PARTITION.code()) { this.metadata.requestUpdate(); } else if (partition.errorCode == Errors.OFFSET_OUT_OF_RANGE.code()) { // TODO: this could be optimized by grouping all out-of-range partitions resetOffset(tp); + } else if (partition.errorCode == Errors.UNKNOWN.code()) { + log.warn("Unknown error fetching data for topic-partition {}", tp); + } else { + throw new IllegalStateException("Unexpected error code " + partition.errorCode + " while fetching data"); } } this.metrics.bytesFetched.record(totalBytes); diff --git a/clients/src/main/java/org/apache/kafka/common/protocol/Errors.java b/clients/src/main/java/org/apache/kafka/common/protocol/Errors.java index 2ec86b2..cb4b50c 100644 --- a/clients/src/main/java/org/apache/kafka/common/protocol/Errors.java +++ b/clients/src/main/java/org/apache/kafka/common/protocol/Errors.java @@ -43,6 +43,9 @@ public enum Errors { new NotLeaderForPartitionException("This server is not the leader for that topic-partition.")), REQUEST_TIMED_OUT(7, new TimeoutException("The request timed out.")), + // TODO: errorCode 8 for BrokerNotAvailable + REPLICA_NOT_AVAILABLE(9, + new ApiException("The replica is not available for the requested topic-partition")), MESSAGE_TOO_LARGE(10, new RecordTooLargeException("The request included a message larger than the max message size the server will accept.")), OFFSET_METADATA_TOO_LARGE(12, @@ -66,7 +69,9 @@ public enum Errors { INVALID_REQUIRED_ACKS(21, new InvalidRequiredAcksException("Produce request specified an invalid value for required acks.")), ILLEGAL_CONSUMER_GENERATION(22, - new ApiException("Specified consumer generation id is not valid.")); + new ApiException("Specified consumer generation id is not valid.")), + NO_OFFSETS_COMMITTED(23, + new ApiException("No offsets have been committed so far.")); private static Map, Errors> classToError = new HashMap, Errors>(); private static Map codeToError = new HashMap(); diff --git a/clients/src/main/java/org/apache/kafka/common/requests/FetchResponse.java b/clients/src/main/java/org/apache/kafka/common/requests/FetchResponse.java index e67c4c8..f020aaa 100644 --- a/clients/src/main/java/org/apache/kafka/common/requests/FetchResponse.java +++ b/clients/src/main/java/org/apache/kafka/common/requests/FetchResponse.java @@ -41,6 +41,17 @@ public class FetchResponse extends AbstractRequestResponse { // partition level field names private static final String PARTITION_KEY_NAME = "partition"; private static final String ERROR_CODE_KEY_NAME = "error_code"; + + /** + * Possible error code: + * + * OFFSET_OUT_OF_RANGE (1) + * UNKNOWN_TOPIC_OR_PARTITION (3) + * NOT_LEADER_FOR_PARTITION (6) + * REPLICA_NOT_AVAILABLE (9) + * UNKNOWN (-1) + */ + private static final String HIGH_WATERMARK_KEY_NAME = "high_watermark"; private static final String RECORD_SET_KEY_NAME = "record_set"; diff --git a/clients/src/main/java/org/apache/kafka/common/requests/HeartbeatResponse.java b/clients/src/main/java/org/apache/kafka/common/requests/HeartbeatResponse.java index 0057496..f548cd0 100644 --- a/clients/src/main/java/org/apache/kafka/common/requests/HeartbeatResponse.java +++ b/clients/src/main/java/org/apache/kafka/common/requests/HeartbeatResponse.java @@ -24,6 +24,12 @@ public class HeartbeatResponse extends AbstractRequestResponse { private static final Schema CURRENT_SCHEMA = ProtoUtils.currentResponseSchema(ApiKeys.HEARTBEAT.id); private static final String ERROR_CODE_KEY_NAME = "error_code"; + /** + * Possible error code: + * + * TODO + */ + private final short errorCode; public HeartbeatResponse(short errorCode) { super(new Struct(CURRENT_SCHEMA)); diff --git a/clients/src/main/java/org/apache/kafka/common/requests/JoinGroupResponse.java b/clients/src/main/java/org/apache/kafka/common/requests/JoinGroupResponse.java index 52b1803..fd9c545 100644 --- a/clients/src/main/java/org/apache/kafka/common/requests/JoinGroupResponse.java +++ b/clients/src/main/java/org/apache/kafka/common/requests/JoinGroupResponse.java @@ -26,6 +26,13 @@ public class JoinGroupResponse extends AbstractRequestResponse { private static final Schema CURRENT_SCHEMA = ProtoUtils.currentResponseSchema(ApiKeys.JOIN_GROUP.id); private static final String ERROR_CODE_KEY_NAME = "error_code"; + + /** + * Possible error code: + * + * TODO + */ + private static final String GENERATION_ID_KEY_NAME = "group_generation_id"; private static final String CONSUMER_ID_KEY_NAME = "consumer_id"; private static final String ASSIGNED_PARTITIONS_KEY_NAME = "assigned_partitions"; diff --git a/clients/src/main/java/org/apache/kafka/common/requests/ListOffsetResponse.java b/clients/src/main/java/org/apache/kafka/common/requests/ListOffsetResponse.java index cfac47a..af704f3 100644 --- a/clients/src/main/java/org/apache/kafka/common/requests/ListOffsetResponse.java +++ b/clients/src/main/java/org/apache/kafka/common/requests/ListOffsetResponse.java @@ -41,6 +41,13 @@ public class ListOffsetResponse extends AbstractRequestResponse { // partition level field names private static final String PARTITION_KEY_NAME = "partition"; private static final String ERROR_CODE_KEY_NAME = "error_code"; + + /** + * Possible error code: + * + * TODO + */ + private static final String OFFSETS_KEY_NAME = "offsets"; private final Map responseData; diff --git a/clients/src/main/java/org/apache/kafka/common/requests/MetadataResponse.java b/clients/src/main/java/org/apache/kafka/common/requests/MetadataResponse.java index 90f3141..36736ec 100644 --- a/clients/src/main/java/org/apache/kafka/common/requests/MetadataResponse.java +++ b/clients/src/main/java/org/apache/kafka/common/requests/MetadataResponse.java @@ -40,11 +40,25 @@ public class MetadataResponse extends AbstractRequestResponse { // topic level field names private static final String TOPIC_ERROR_CODE_KEY_NAME = "topic_error_code"; + + /** + * Possible error code: + * + * TODO + */ + private static final String TOPIC_KEY_NAME = "topic"; private static final String PARTITION_METADATA_KEY_NAME = "partition_metadata"; // partition level field names private static final String PARTITION_ERROR_CODE_KEY_NAME = "partition_error_code"; + + /** + * Possible error code: + * + * TODO + */ + private static final String PARTITION_KEY_NAME = "partition_id"; private static final String LEADER_KEY_NAME = "leader"; private static final String REPLICAS_KEY_NAME = "replicas"; diff --git a/clients/src/main/java/org/apache/kafka/common/requests/OffsetCommitResponse.java b/clients/src/main/java/org/apache/kafka/common/requests/OffsetCommitResponse.java index 4d3b9ec..70844d6 100644 --- a/clients/src/main/java/org/apache/kafka/common/requests/OffsetCommitResponse.java +++ b/clients/src/main/java/org/apache/kafka/common/requests/OffsetCommitResponse.java @@ -38,6 +38,12 @@ public class OffsetCommitResponse extends AbstractRequestResponse { private static final String PARTITION_KEY_NAME = "partition"; private static final String ERROR_CODE_KEY_NAME = "error_code"; + /** + * Possible error code: + * + * TODO + */ + private final Map responseData; public OffsetCommitResponse(Map responseData) { diff --git a/clients/src/main/java/org/apache/kafka/common/requests/OffsetFetchResponse.java b/clients/src/main/java/org/apache/kafka/common/requests/OffsetFetchResponse.java index edbed58..54100a9 100644 --- a/clients/src/main/java/org/apache/kafka/common/requests/OffsetFetchResponse.java +++ b/clients/src/main/java/org/apache/kafka/common/requests/OffsetFetchResponse.java @@ -41,6 +41,15 @@ public class OffsetFetchResponse extends AbstractRequestResponse { private static final String METADATA_KEY_NAME = "metadata"; private static final String ERROR_CODE_KEY_NAME = "error_code"; + /** + * Possible error code: + * + * UNKNOWN_TOPIC_OR_PARTITION (3) + * OFFSET_LOAD_IN_PROGRESS (14) + * NOT_COORDINATOR_FOR_CONSUMER (16) + * NO_OFFSETS_COMMITTED (23) + */ + private final Map responseData; public static final class PartitionData { 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 a00dcdf..4b67f70 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 @@ -37,6 +37,13 @@ public class ProduceResponse extends AbstractRequestResponse { // partition level field names private static final String PARTITION_KEY_NAME = "partition"; private static final String ERROR_CODE_KEY_NAME = "error_code"; + + /** + * Possible error code: + * + * TODO + */ + private static final String BASE_OFFSET_KEY_NAME = "base_offset"; private final Map responses; diff --git a/core/src/main/scala/kafka/common/ErrorMapping.scala b/core/src/main/scala/kafka/common/ErrorMapping.scala index eedc2f5..eb1eb4a 100644 --- a/core/src/main/scala/kafka/common/ErrorMapping.scala +++ b/core/src/main/scala/kafka/common/ErrorMapping.scala @@ -49,6 +49,9 @@ object ErrorMapping { val MessageSetSizeTooLargeCode: Short = 18 val NotEnoughReplicasCode : Short = 19 val NotEnoughReplicasAfterAppendCode: Short = 20 + // 21: InvalidRequiredAcks + // 22: IllegalConsumerGeneration + val NoOffsetsCommittedCode: Short = 23 private val exceptionToCode = Map[Class[Throwable], Short]( @@ -70,7 +73,8 @@ object ErrorMapping { classOf[InvalidTopicException].asInstanceOf[Class[Throwable]] -> InvalidTopicCode, classOf[MessageSetSizeTooLargeException].asInstanceOf[Class[Throwable]] -> MessageSetSizeTooLargeCode, classOf[NotEnoughReplicasException].asInstanceOf[Class[Throwable]] -> NotEnoughReplicasCode, - classOf[NotEnoughReplicasAfterAppendException].asInstanceOf[Class[Throwable]] -> NotEnoughReplicasAfterAppendCode + classOf[NotEnoughReplicasAfterAppendException].asInstanceOf[Class[Throwable]] -> NotEnoughReplicasAfterAppendCode, + classOf[NoOffsetsCommittedException].asInstanceOf[Class[Throwable]] -> NoOffsetsCommittedCode ).withDefaultValue(UnknownCode) /* invert the mapping */ diff --git a/core/src/main/scala/kafka/common/NoOffsetsCommittedException.scala b/core/src/main/scala/kafka/common/NoOffsetsCommittedException.scala new file mode 100644 index 0000000..2a68e87 --- /dev/null +++ b/core/src/main/scala/kafka/common/NoOffsetsCommittedException.scala @@ -0,0 +1,27 @@ +/* + * 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 kafka.common + +/** + * Number of insync replicas for the partition is lower than min.insync.replicas + * This exception is raised when the low ISR size is discovered *after* the message + * was already appended to the log. Producer retries will cause duplicates. + */ +class NoOffsetsCommittedException(message: String) extends RuntimeException(message) { + def this() = this(null) +} \ No newline at end of file diff --git a/core/src/main/scala/kafka/common/OffsetMetadataAndError.scala b/core/src/main/scala/kafka/common/OffsetMetadataAndError.scala index 4cabffe..1584a92 100644 --- a/core/src/main/scala/kafka/common/OffsetMetadataAndError.scala +++ b/core/src/main/scala/kafka/common/OffsetMetadataAndError.scala @@ -48,7 +48,7 @@ case class OffsetMetadataAndError(offset: Long, } object OffsetMetadataAndError { - val NoOffset = OffsetMetadataAndError(OffsetAndMetadata.InvalidOffset, OffsetAndMetadata.NoMetadata, ErrorMapping.NoError) + val NoOffset = OffsetMetadataAndError(OffsetAndMetadata.InvalidOffset, OffsetAndMetadata.NoMetadata, ErrorMapping.NoOffsetsCommittedCode) val OffsetsLoading = OffsetMetadataAndError(OffsetAndMetadata.InvalidOffset, OffsetAndMetadata.NoMetadata, ErrorMapping.OffsetsLoadInProgressCode) val NotOffsetManagerForGroup = OffsetMetadataAndError(OffsetAndMetadata.InvalidOffset, OffsetAndMetadata.NoMetadata, ErrorMapping.NotCoordinatorForConsumerCode) val UnknownTopicOrPartition = OffsetMetadataAndError(OffsetAndMetadata.InvalidOffset, OffsetAndMetadata.NoMetadata, ErrorMapping.UnknownTopicOrPartitionCode) diff --git a/core/src/test/resources/log4j.properties b/core/src/test/resources/log4j.properties index 9973dad..6941125 100644 --- a/core/src/test/resources/log4j.properties +++ b/core/src/test/resources/log4j.properties @@ -12,14 +12,14 @@ # 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. -log4j.rootLogger=TRACE, stdout +log4j.rootLogger=DEBUG, stdout log4j.appender.stdout=org.apache.log4j.ConsoleAppender log4j.appender.stdout.layout=org.apache.log4j.PatternLayout log4j.appender.stdout.layout.ConversionPattern=[%d] %p %m (%c:%L)%n -log4j.logger.kafka=TRACE -log4j.logger.org.apache.kafka=TRACE +log4j.logger.kafka=DEBUG +log4j.logger.org.apache.kafka=DEBUG # zkclient can be verbose, during debugging it is common to adjust is separately log4j.logger.org.I0Itec.zkclient.ZkClient=WARN diff --git a/core/src/test/scala/integration/kafka/api/ConsumerTest.scala b/core/src/test/scala/integration/kafka/api/ConsumerTest.scala index db4aa6d..0a4e12d 100644 --- a/core/src/test/scala/integration/kafka/api/ConsumerTest.scala +++ b/core/src/test/scala/integration/kafka/api/ConsumerTest.scala @@ -139,7 +139,7 @@ class ConsumerTest extends IntegrationTestHarness with Logging { consumeRecords(this.consumers(0), 5, 0) assertEquals("After consuming 5 records, position should be 5", 5L, this.consumers(0).position(tp)) this.consumers(0).commit(CommitType.SYNC) - assertEquals("Committed offset should be returned", 5L, this.consumers(0).committed(tp)); + assertEquals("Committed offset should be returned", 5L, this.consumers(0).committed(tp)) sendRecords(1) diff --git a/core/src/test/scala/integration/kafka/api/IntegrationTestHarness.scala b/core/src/test/scala/integration/kafka/api/IntegrationTestHarness.scala index c58b7a9..0cc37fb 100644 --- a/core/src/test/scala/integration/kafka/api/IntegrationTestHarness.scala +++ b/core/src/test/scala/integration/kafka/api/IntegrationTestHarness.scala @@ -19,14 +19,11 @@ package kafka.api import org.apache.kafka.clients.producer.ProducerConfig import org.apache.kafka.clients.consumer.ConsumerConfig -import org.scalatest.junit.JUnit3Suite -import collection._ import kafka.utils.TestUtils import java.util.Properties -import java.util.Arrays import org.apache.kafka.clients.consumer.KafkaConsumer import org.apache.kafka.clients.producer.KafkaProducer -import kafka.server.KafkaConfig +import kafka.server.{OffsetManager, KafkaConfig} import kafka.integration.KafkaServerTestHarness import scala.collection.mutable.Buffer @@ -63,12 +60,12 @@ trait IntegrationTestHarness extends KafkaServerTestHarness { for(i <- 0 until consumerCount) consumers += new KafkaConsumer(consumerConfig) - // make sure the consumer's metadata has been refreshed with the full broker list - for(i <- 0 until consumerCount) { - while consumers(i) - } - - + // create the consumer offset topic + TestUtils.createTopic(zkClient, OffsetManager.OffsetsTopicName, + serverConfig.getProperty("offsets.topic.num.partitions").toInt, + serverConfig.getProperty("offsets.topic.replication.factor").toInt, + servers, + servers(0).offsetManager.offsetsTopicConfig) } override def tearDown() { -- 1.7.12.4 From dcf2b2008e7a00753bef0303df3274dad075ff43 Mon Sep 17 00:00:00 2001 From: Guozhang Wang Date: Mon, 2 Mar 2015 16:16:13 -0800 Subject: [PATCH 09/11] fixed tests --- .../kafka/clients/consumer/KafkaConsumer.java | 8 +- .../clients/consumer/internals/Coordinator.java | 20 ++--- .../clients/consumer/internals/FetchManager.java | 4 +- .../consumer/internals/SubscriptionState.java | 18 ++--- .../scala/integration/kafka/api/ConsumerTest.scala | 93 +++++++++++----------- .../kafka/integration/KafkaServerTestHarness.scala | 2 +- .../scala/unit/kafka/server/OffsetCommitTest.scala | 2 +- .../test/scala/unit/kafka/utils/TestUtils.scala | 22 +++++ 8 files changed, 98 insertions(+), 71 deletions(-) diff --git a/clients/src/main/java/org/apache/kafka/clients/consumer/KafkaConsumer.java b/clients/src/main/java/org/apache/kafka/clients/consumer/KafkaConsumer.java index 3d1c9e1..ecae386 100644 --- a/clients/src/main/java/org/apache/kafka/clients/consumer/KafkaConsumer.java +++ b/clients/src/main/java/org/apache/kafka/clients/consumer/KafkaConsumer.java @@ -617,7 +617,7 @@ public class KafkaConsumer implements Consumer { if (subscriptions.partitionsAutoAssigned()) { // rebalance to get partition assignment - if (subscriptions.needsPartitionAssignment()) { + if (subscriptions.partitionAssignmentNeeded()) { // execute the user's callback before rebalance log.debug("Revoking previously assigned partitions {}", this.subscriptions.assignedPartitions()); try { @@ -632,7 +632,7 @@ public class KafkaConsumer implements Consumer { new ArrayList(this.subscriptions.subscribedTopics()), now)); // execute the user's callback after rebalance - log.debug("Revoking previously assigned partitions {}", this.subscriptions.assignedPartitions()); + log.debug("Setting newly assigned partitions {}", this.subscriptions.assignedPartitions()); try { this.rebalanceCallback.onPartitionsAssigned(this, this.subscriptions.assignedPartitions()); } catch (Exception e) { @@ -698,7 +698,7 @@ public class KafkaConsumer implements Consumer { // commit the offsets with the coordinator boolean syncCommit = commitType.equals(CommitType.SYNC); if (!syncCommit) - this.subscriptions.commitInProgress(); + this.subscriptions.needRefreshCommits(); coordinator.commitOffsets(offsets, syncCommit, now); } @@ -892,7 +892,7 @@ public class KafkaConsumer implements Consumer { // we only need to fetch latest committed offset from coordinator if there // is some commit process in progress, otherwise our current // committed cache is up-to-date - if (subscriptions.needsSyncCommits()) { + if (subscriptions.refreshCommitsNeeded()) { // contact coordinator to fetch committed offsets Map offsets = coordinator.fetchOffsets(partitions, now); diff --git a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/Coordinator.java b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/Coordinator.java index 537c266..d26a829 100644 --- a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/Coordinator.java +++ b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/Coordinator.java @@ -182,6 +182,9 @@ public final class Coordinator { this.consumerId = response.consumerId(); this.heartbeat.receivedResponse(now); + // set the flag to refresh last committed offsets + this.subscriptions.needRefreshCommits(); + log.debug("Joined group: {}", response); // return assigned partitions @@ -245,7 +248,7 @@ public final class Coordinator { public Map fetchOffsets(Set partitions, long now) { log.debug("Fetching committed offsets for partitions: " + Utils.join(partitions, ", ")); - while(true) { + while (true) { // construct the request OffsetFetchRequest request = new OffsetFetchRequest(this.groupId, new ArrayList(partitions)); @@ -346,9 +349,7 @@ public final class Coordinator { // if the coordinator connection has failed, we need to // break the inner loop to re-discover the coordinator - if (this.consumerCoordinator == null) { - break; - } else if (this.client.connectionFailed(this.consumerCoordinator)) { + if (this.client.connectionFailed(this.consumerCoordinator)) { log.debug("Coordinator connection failed. Attempting to re-discover."); coordinatorDead(); break; @@ -362,9 +363,11 @@ public final class Coordinator { * Mark the current coordinator as dead. */ private void coordinatorDead() { - log.info("Marking the coordinator {} dead.", this.consumerCoordinator.id()); - this.heartbeat.markDead(); - this.consumerCoordinator = null; + if (this.consumerCoordinator != null) { + log.info("Marking the coordinator {} dead.", this.consumerCoordinator.id()); + this.heartbeat.markDead(); + this.consumerCoordinator = null; + } } /** @@ -427,8 +430,7 @@ public final class Coordinator { response.request().request().destination()); // mark the coordinator as dead - if (this.consumerCoordinator != null) - coordinatorDead(); + coordinatorDead(); } /** diff --git a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/FetchManager.java b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/FetchManager.java index 346ce00..c8972de 100644 --- a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/FetchManager.java +++ b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/FetchManager.java @@ -48,7 +48,7 @@ import java.util.*; */ public class FetchManager { - private static final Logger log = LoggerFactory.getLogger(Coordinator.class); + private static final Logger log = LoggerFactory.getLogger(FetchManager.class); private static final long EARLIEST_OFFSET_TIMESTAMP = -2L; private static final long LATEST_OFFSET_TIMESTAMP = -1L; @@ -124,7 +124,7 @@ public class FetchManager { * @return The fetched records per partition */ public Map>> fetchedRecords() { - if (this.subscriptions.needsPartitionAssignment()) { + if (this.subscriptions.partitionAssignmentNeeded()) { return Collections.emptyMap(); } else { Map>> drained = new HashMap>>(); diff --git a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/SubscriptionState.java b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/SubscriptionState.java index d9df339..cee7541 100644 --- a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/SubscriptionState.java +++ b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/SubscriptionState.java @@ -46,8 +46,8 @@ public class SubscriptionState { /* do we need to request a partition assignment from the coordinator? */ private boolean needsPartitionAssignment; - /* is there an asynchronous commit request on-going with the coordinator? */ - private boolean commitInProgress; + /* do we need to request the latest committed offsets from the coordinator? */ + private boolean needsFetchCommittedOffsets; public SubscriptionState() { this.subscribedTopics = new HashSet(); @@ -57,7 +57,7 @@ public class SubscriptionState { this.fetched = new HashMap(); this.committed = new HashMap(); this.needsPartitionAssignment = false; - this.commitInProgress = true; // initialize to true for the consumers to fetch offset upon starting up + this.needsFetchCommittedOffsets = true; // initialize to true for the consumers to fetch offset upon starting up } public void subscribe(String topic) { @@ -127,19 +127,19 @@ public class SubscriptionState { public void committed(TopicPartition tp, long offset) { this.committed.put(tp, offset); - this.commitInProgress = false; + this.needsFetchCommittedOffsets = false; } public Long committed(TopicPartition tp) { return this.committed.get(tp); } - public void commitInProgress() { - this.commitInProgress = true; + public void needRefreshCommits() { + this.needsFetchCommittedOffsets = true; } - public boolean needsSyncCommits() { - return this.commitInProgress; + public boolean refreshCommitsNeeded() { + return this.needsFetchCommittedOffsets; } public void seek(TopicPartition tp, long offset) { @@ -179,7 +179,7 @@ public class SubscriptionState { return copy; } - public boolean needsPartitionAssignment() { + public boolean partitionAssignmentNeeded() { return this.needsPartitionAssignment; } diff --git a/core/src/test/scala/integration/kafka/api/ConsumerTest.scala b/core/src/test/scala/integration/kafka/api/ConsumerTest.scala index 0a4e12d..4710141 100644 --- a/core/src/test/scala/integration/kafka/api/ConsumerTest.scala +++ b/core/src/test/scala/integration/kafka/api/ConsumerTest.scala @@ -29,8 +29,7 @@ import scala.collection.JavaConversions._ import java.util.ArrayList import java.util.Arrays import org.junit.Assert._ -import kafka.utils.TestUtils -import kafka.utils.Logging +import kafka.utils.{ShutdownableThread, TestUtils, Logging} import kafka.server.OffsetManager /** @@ -57,13 +56,12 @@ class ConsumerTest extends IntegrationTestHarness with Logging { override def setUp() { super.setUp() - // this will trigger the creation of the consumer offsets topic - this.consumers(0).partitionsFor(OffsetManager.OffsetsTopicName) - } - def testSimpleConsumption() { + // create the test topic with all the brokers as replicas TestUtils.createTopic(this.zkClient, topic, 1, serverCount, this.servers) + } + def testSimpleConsumption() { val numRecords = 10000 sendRecords(numRecords) @@ -76,16 +74,12 @@ class ConsumerTest extends IntegrationTestHarness with Logging { } def testAutoOffsetReset() { - TestUtils.createTopic(this.zkClient, topic, 1, serverCount, this.servers) - sendRecords(1) this.consumers(0).subscribe(tp) consumeRecords(this.consumers(0), numRecords = 1, startingOffset = 0) } def testSeek() { - TestUtils.createTopic(this.zkClient, topic, 1, serverCount, this.servers) - val consumer = this.consumers(0) val totalRecords = 50L sendRecords(totalRecords.toInt) @@ -93,7 +87,7 @@ class ConsumerTest extends IntegrationTestHarness with Logging { consumer.seekToEnd(tp) assertEquals(totalRecords, consumer.position(tp)) - assertFalse(consumer.poll(totalRecords).iterator().hasNext()) + assertFalse(consumer.poll(totalRecords).iterator().hasNext) consumer.seekToBeginning(tp) assertEquals(0, consumer.position(tp), 0) @@ -106,18 +100,12 @@ class ConsumerTest extends IntegrationTestHarness with Logging { } def testGroupConsumption() { - TestUtils.createTopic(this.zkClient, topic, 1, serverCount, this.servers) - // we need to do this test with only one server since we have the hack join group - // that just assigns the partition hosted on the local machine (with two we might get the wrong machine - this.servers.last.shutdown() - this.servers.head.shutdown() sendRecords(10) this.consumers(0).subscribe(topic) consumeRecords(this.consumers(0), numRecords = 1, startingOffset = 0) } def testPositionAndCommit() { - TestUtils.createTopic(this.zkClient, topic, 1, serverCount, this.servers) sendRecords(5) // committed() on a partition with no committed offset throws an exception @@ -150,61 +138,66 @@ class ConsumerTest extends IntegrationTestHarness with Logging { def testPartitionsFor() { val numParts = 2 - TestUtils.createTopic(this.zkClient, topic, numParts, 1, this.servers) - val parts = this.consumers(0).partitionsFor(topic) + TestUtils.createTopic(this.zkClient, "part-test", numParts, 1, this.servers) + val parts = this.consumers(0).partitionsFor("part-test") assertNotNull(parts) assertEquals(2, parts.length) - assertNull(this.consumers(0).partitionsFor("non-existant-topic")) + assertNull(this.consumers(0).partitionsFor("non-exist-topic")) } - def testConsumptionWithBrokerFailures() = consumeWithBrokerFailures(numRecords = 1000) + def testConsumptionWithBrokerFailures() = consumeWithBrokerFailures(5) /* * 1. Produce a bunch of messages * 2. Then consume the messages while killing and restarting brokers at random */ - def consumeWithBrokerFailures(numRecords: Int) { - TestUtils.createTopic(this.zkClient, topic, 1, serverCount, this.servers) + def consumeWithBrokerFailures(numIters: Int) { + val numRecords = 1000 sendRecords(numRecords) this.producers.map(_.close) + var consumed = 0 val consumer = this.consumers(0) consumer.subscribe(topic) - while (consumed < numRecords) { - // check that we are getting the messages in order - for (record <- consumer.poll(200)) { + + val scheduler = new BounceBrokerScheduler(numIters) + scheduler.start() + + while (scheduler.isRunning.get()) { + for (record <- consumer.poll(100)) { assertEquals(consumed.toLong, record.offset()) consumed += 1 } consumer.commit(CommitType.SYNC) - /* restart any dead brokers, and kill a broker (with probability 1/3) */ - restartDeadBrokers() - if (TestUtils.random.nextInt(3) == 0) { - info("Killing broker") - killRandomBroker() + if (consumed == numRecords) { + consumer.seekToBeginning() + consumed = 0 } } + scheduler.shutdown() } - def testSeekAndCommitWithBrokerFailures() = seekAndCommitWithBrokerFailures(20) + def testSeekAndCommitWithBrokerFailures() = seekAndCommitWithBrokerFailures(5) def seekAndCommitWithBrokerFailures(numIters: Int) { - // create a topic and send it some data val numRecords = 1000 - TestUtils.createTopic(this.zkClient, topic, 1, serverCount, this.servers) sendRecords(numRecords) this.producers.map(_.close) val consumer = this.consumers(0) consumer.subscribe(tp) consumer.seek(tp, 0) - for (iter <- 0 until numIters) { - val coin = TestUtils.random.nextInt(4) + + val scheduler = new BounceBrokerScheduler(numIters) + scheduler.start() + + while(scheduler.isRunning.get()) { + val coin = TestUtils.random.nextInt(3) if (coin == 0) { info("Seeking to end of log") consumer.seekToEnd() - assertEquals(1000.toLong, consumer.position(tp)) + assertEquals(numRecords.toLong, consumer.position(tp)) } else if (coin == 1) { val pos = TestUtils.random.nextInt(numRecords).toLong info("Seeking to " + pos) @@ -214,17 +207,11 @@ class ConsumerTest extends IntegrationTestHarness with Logging { info("Committing offset.") consumer.commit(CommitType.SYNC) assertEquals(consumer.position(tp), consumer.committed(tp)) - } else { - restartDeadBrokers() - killRandomBroker() } } } def testPartitionReassignmentCallback() { - // create the topic with replication - TestUtils.createTopic(this.zkClient, topic, 1, serverCount, this.servers) - val callback = new TestConsumerReassignmentCallback() this.consumerConfig.setProperty(ConsumerConfig.SESSION_TIMEOUT_MS_CONFIG, "200"); // timeout quickly to avoid slow test val consumer0 = new KafkaConsumer(this.consumerConfig, callback, new ByteArrayDeserializer(), new ByteArrayDeserializer()) @@ -253,8 +240,8 @@ class ConsumerTest extends IntegrationTestHarness with Logging { consumer0.close() } - - class TestConsumerReassignmentCallback extends ConsumerRebalanceCallback { + + private class TestConsumerReassignmentCallback extends ConsumerRebalanceCallback { var callsToAssigned = 0 var callsToRevoked = 0 def onPartitionsAssigned(consumer: Consumer[_,_], partitions: java.util.Collection[TopicPartition]) { @@ -267,6 +254,22 @@ class ConsumerTest extends IntegrationTestHarness with Logging { } } + private class BounceBrokerScheduler(val numIters: Int) extends ShutdownableThread("daemon-bounce-broker", false) + { + var iter: Int = 0 + + override def doWork(): Unit = { + killRandomBroker() + restartDeadBrokers() + + iter += 1 + if (iter == numIters) + initiateShutdown() + else + Thread.sleep(500) + } + } + private def sendRecords(numRecords: Int) { val futures = (0 until numRecords).map { i => this.producers(0).send(new ProducerRecord(topic, part, i.toString.getBytes, i.toString.getBytes)) diff --git a/core/src/test/scala/unit/kafka/integration/KafkaServerTestHarness.scala b/core/src/test/scala/unit/kafka/integration/KafkaServerTestHarness.scala index dc0512b..062790f 100644 --- a/core/src/test/scala/unit/kafka/integration/KafkaServerTestHarness.scala +++ b/core/src/test/scala/unit/kafka/integration/KafkaServerTestHarness.scala @@ -75,7 +75,7 @@ trait KafkaServerTestHarness extends JUnit3Suite with ZooKeeperTestHarness { */ def restartDeadBrokers() { for(i <- 0 until servers.length if !alive(i)) { - servers(i) = TestUtils.createServer(configs(i)) + servers(i).startup() alive(i) = true } } diff --git a/core/src/test/scala/unit/kafka/server/OffsetCommitTest.scala b/core/src/test/scala/unit/kafka/server/OffsetCommitTest.scala index a2bb885..1df1b0d 100644 --- a/core/src/test/scala/unit/kafka/server/OffsetCommitTest.scala +++ b/core/src/test/scala/unit/kafka/server/OffsetCommitTest.scala @@ -159,7 +159,7 @@ class OffsetCommitTest extends JUnit3Suite with ZooKeeperTestHarness { assertEquals(ErrorMapping.NoError, fetchResponse.requestInfo.get(TopicAndPartition(topic3, 0)).get.error) assertEquals(ErrorMapping.NoError, fetchResponse.requestInfo.get(TopicAndPartition(topic2, 1)).get.error) assertEquals(ErrorMapping.UnknownTopicOrPartitionCode, fetchResponse.requestInfo.get(TopicAndPartition(topic3, 1)).get.error) - assertEquals(ErrorMapping.NoError, fetchResponse.requestInfo.get(TopicAndPartition(topic4, 0)).get.error) + assertEquals(ErrorMapping.NoOffsetsCommittedCode, fetchResponse.requestInfo.get(TopicAndPartition(topic4, 0)).get.error) assertEquals(ErrorMapping.UnknownTopicOrPartitionCode, fetchResponse.requestInfo.get(TopicAndPartition(topic5, 0)).get.error) assertEquals(OffsetMetadataAndError.UnknownTopicOrPartition, fetchResponse.requestInfo.get(TopicAndPartition(topic3, 1)).get) assertEquals(OffsetMetadataAndError.NoOffset, fetchResponse.requestInfo.get(TopicAndPartition(topic4, 0)).get) diff --git a/core/src/test/scala/unit/kafka/utils/TestUtils.scala b/core/src/test/scala/unit/kafka/utils/TestUtils.scala index 6ce1807..52c7920 100644 --- a/core/src/test/scala/unit/kafka/utils/TestUtils.scala +++ b/core/src/test/scala/unit/kafka/utils/TestUtils.scala @@ -48,6 +48,7 @@ import org.apache.kafka.clients.producer.KafkaProducer import collection.Iterable import scala.collection.Map +import org.apache.kafka.clients.consumer.KafkaConsumer /** * Utility functions to help with testing @@ -407,6 +408,27 @@ object TestUtils extends Logging { } /** + * Create a new consumer with a few pre-configured properties. + */ + def createNewConsumer(brokerList: String, + groupId: String, + autoOffsetReset: String = "earliest", + partitionFetchSize: Long = 4096L) : KafkaConsumer[Array[Byte],Array[Byte]] = { + import org.apache.kafka.clients.consumer.ConsumerConfig + + val consumerProps= new Properties() + consumerProps.put(ConsumerConfig.BOOTSTRAP_SERVERS_CONFIG, brokerList) + consumerProps.put(ConsumerConfig.GROUP_ID_CONFIG, groupId) + consumerProps.put(ConsumerConfig.AUTO_OFFSET_RESET_CONFIG, autoOffsetReset) + consumerProps.put(ConsumerConfig.MAX_PARTITION_FETCH_BYTES_CONFIG, partitionFetchSize.toString) + consumerProps.put(ConsumerConfig.RETRY_BACKOFF_MS_CONFIG, "100") + consumerProps.put(ConsumerConfig.RECONNECT_BACKOFF_MS_CONFIG, "200") + consumerProps.put(ConsumerConfig.KEY_DESERIALIZER_CLASS_CONFIG, "org.apache.kafka.common.serialization.ByteArrayDeserializer") + consumerProps.put(ConsumerConfig.VALUE_DESERIALIZER_CLASS_CONFIG, "org.apache.kafka.common.serialization.ByteArrayDeserializer") + return new KafkaConsumer[Array[Byte],Array[Byte]](consumerProps) + } + + /** * Create a default producer config properties map with the given metadata broker list */ def getProducerConfig(brokerList: String): Properties = { -- 1.7.12.4 From 91a2eef4b08f6d1cb6380763cd377ac96384c614 Mon Sep 17 00:00:00 2001 From: Guozhang Wang Date: Mon, 2 Mar 2015 16:20:54 -0800 Subject: [PATCH 10/11] clean imports --- .../src/test/scala/integration/kafka/api/ConsumerTest.scala | 13 +++++++------ .../integration/kafka/api/ProducerFailureHandlingTest.scala | 2 +- 2 files changed, 8 insertions(+), 7 deletions(-) diff --git a/core/src/test/scala/integration/kafka/api/ConsumerTest.scala b/core/src/test/scala/integration/kafka/api/ConsumerTest.scala index 4710141..ef59ed9 100644 --- a/core/src/test/scala/integration/kafka/api/ConsumerTest.scala +++ b/core/src/test/scala/integration/kafka/api/ConsumerTest.scala @@ -12,7 +12,6 @@ */ package kafka.api -import org.apache.kafka.common.KafkaException import org.apache.kafka.clients.producer.ProducerConfig import org.apache.kafka.clients.producer.ProducerRecord import org.apache.kafka.clients.consumer.Consumer @@ -24,14 +23,16 @@ import org.apache.kafka.clients.consumer.CommitType import org.apache.kafka.common.serialization.ByteArrayDeserializer import org.apache.kafka.common.TopicPartition import org.apache.kafka.clients.consumer.NoOffsetForPartitionException -import scala.collection.mutable.Buffer -import scala.collection.JavaConversions._ -import java.util.ArrayList -import java.util.Arrays -import org.junit.Assert._ + import kafka.utils.{ShutdownableThread, TestUtils, Logging} import kafka.server.OffsetManager +import java.util.ArrayList +import org.junit.Assert._ + +import scala.collection.JavaConversions._ + + /** * Integration tests for the new consumer that cover basic usage as well as server failures */ diff --git a/core/src/test/scala/integration/kafka/api/ProducerFailureHandlingTest.scala b/core/src/test/scala/integration/kafka/api/ProducerFailureHandlingTest.scala index ba48a63..13418ab 100644 --- a/core/src/test/scala/integration/kafka/api/ProducerFailureHandlingTest.scala +++ b/core/src/test/scala/integration/kafka/api/ProducerFailureHandlingTest.scala @@ -15,7 +15,7 @@ * limitations under the License. */ -package kafka.api.test +package kafka.api import org.junit.Test import org.junit.Assert._ -- 1.7.12.4 From 0c9ebf92b983c23a7ce648ac8b1b1259482fd280 Mon Sep 17 00:00:00 2001 From: Guozhang Wang Date: Mon, 2 Mar 2015 16:44:18 -0800 Subject: [PATCH 11/11] log properties --- .../org/apache/kafka/clients/consumer/internals/FetchManager.java | 8 +++++++- core/src/test/resources/log4j.properties | 6 +++--- 2 files changed, 10 insertions(+), 4 deletions(-) diff --git a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/FetchManager.java b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/FetchManager.java index c8972de..ecf4d21 100644 --- a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/FetchManager.java +++ b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/FetchManager.java @@ -41,7 +41,13 @@ import org.slf4j.Logger; import org.slf4j.LoggerFactory; import java.nio.ByteBuffer; -import java.util.*; +import java.util.ArrayList; +import java.util.Collections; +import java.util.HashMap; +import java.util.LinkedList; +import java.util.List; +import java.util.Map; + /** * This class manage the fetching process with the brokers. diff --git a/core/src/test/resources/log4j.properties b/core/src/test/resources/log4j.properties index 6941125..1b7d5d8 100644 --- a/core/src/test/resources/log4j.properties +++ b/core/src/test/resources/log4j.properties @@ -12,14 +12,14 @@ # 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. -log4j.rootLogger=DEBUG, stdout +log4j.rootLogger=OFF, stdout log4j.appender.stdout=org.apache.log4j.ConsoleAppender log4j.appender.stdout.layout=org.apache.log4j.PatternLayout log4j.appender.stdout.layout.ConversionPattern=[%d] %p %m (%c:%L)%n -log4j.logger.kafka=DEBUG -log4j.logger.org.apache.kafka=DEBUG +log4j.logger.kafka=ERROR +log4j.logger.org.apache.kafka=ERROR # zkclient can be verbose, during debugging it is common to adjust is separately log4j.logger.org.I0Itec.zkclient.ZkClient=WARN -- 1.7.12.4