From b10e55ac5c83d0a356f79b0325d0dd8cefe00a42 Mon Sep 17 00:00:00 2001 From: Dong Lin Date: Sun, 2 Apr 2017 17:46:34 -0700 Subject: [PATCH 01/11] KAFKA-4763; Handle disk failure for JBOD (KIP-112) --- .../apache/kafka/common/PartitionInfo.java | 20 +- .../common/errors/KafkaStorageException.java | 33 +++ .../apache/kafka/common/protocol/Errors.java | 10 +- .../kafka/common/protocol/Protocol.java | 95 ++++++- .../kafka/common/record/FileRecords.java | 7 + .../common/requests/LeaderAndIsrRequest.java | 7 +- .../common/requests/MetadataResponse.java | 54 ++-- .../kafka/common/requests/PartitionState.java | 8 +- .../requests/UpdateMetadataRequest.java | 71 ++++- .../consumer/internals/FetcherTest.java | 3 +- .../kafka/common/PartitionInfoTest.java | 15 +- .../common/requests/RequestResponseTest.java | 18 +- .../main/scala/kafka/api/LeaderAndIsr.scala | 17 +- .../main/scala/kafka/cluster/Partition.scala | 22 +- .../controller/ControllerChannelManager.scala | 37 +-- .../kafka/controller/ControllerState.scala | 11 +- .../kafka/controller/KafkaController.scala | 145 +++++++++- .../controller/PartitionLeaderSelector.scala | 9 +- .../controller/PartitionStateMachine.scala | 12 +- .../controller/ReplicaStateMachine.scala | 8 +- .../controller/TopicDeletionManager.scala | 2 +- core/src/main/scala/kafka/log/Log.scala | 10 + .../src/main/scala/kafka/log/LogCleaner.scala | 101 +++---- .../scala/kafka/log/LogCleanerManager.scala | 66 ++++- .../src/main/scala/kafka/log/LogManager.scala | 269 ++++++++++++------ .../src/main/scala/kafka/log/LogSegment.scala | 7 + .../main/scala/kafka/server/KafkaApis.scala | 57 ++-- .../main/scala/kafka/server/KafkaServer.scala | 30 +- .../scala/kafka/server/MetadataCache.scala | 20 +- .../kafka/server/ReplicaFetcherThread.scala | 15 +- .../scala/kafka/server/ReplicaManager.scala | 182 ++++++++---- .../server/checkpoints/CheckpointFile.scala | 9 +- .../main/scala/kafka/utils/DiskUtils.scala | 58 ++++ core/src/main/scala/kafka/utils/ZkUtils.scala | 7 +- .../kafka/api/AuthorizerIntegrationTest.scala | 6 +- .../kafka/api/TransactionsTest.scala | 4 +- .../AbstractLogCleanerIntegrationTest.scala | 4 +- .../kafka/log/LogCleanerIntegrationTest.scala | 2 +- .../kafka/log/LogCleanerManagerTest.scala | 4 +- .../scala/unit/kafka/log/LogManagerTest.scala | 18 +- .../server/HighwatermarkPersistenceTest.scala | 12 +- .../unit/kafka/server/ISRExpirationTest.scala | 15 +- .../kafka/server/LeaderElectionTest.scala | 2 +- .../unit/kafka/server/MetadataCacheTest.scala | 18 +- .../server/ReplicaManagerQuotasTest.scala | 3 + .../kafka/server/ReplicaManagerTest.scala | 59 ++-- .../unit/kafka/server/RequestQuotaTest.scala | 5 +- .../unit/kafka/server/SimpleFetchTest.scala | 5 + .../epoch/OffsetsForLeaderEpochTest.scala | 21 +- .../scala/unit/kafka/utils/TestUtils.scala | 12 +- .../utils/IntegrationTestUtils.java | 8 +- .../internals/InternalTopicManagerTest.java | 2 +- tests/kafkatest/services/kafka/config.py | 2 +- .../services/kafka/config_property.py | 4 + tests/kafkatest/services/kafka/kafka.py | 61 +++- .../tests/core/log_dir_failure_test.py | 177 ++++++++++++ 56 files changed, 1396 insertions(+), 483 deletions(-) create mode 100644 clients/src/main/java/org/apache/kafka/common/errors/KafkaStorageException.java create mode 100644 core/src/main/scala/kafka/utils/DiskUtils.scala create mode 100644 tests/kafkatest/tests/core/log_dir_failure_test.py diff --git a/clients/src/main/java/org/apache/kafka/common/PartitionInfo.java b/clients/src/main/java/org/apache/kafka/common/PartitionInfo.java index b35111696373..b101a9c87ede 100644 --- a/clients/src/main/java/org/apache/kafka/common/PartitionInfo.java +++ b/clients/src/main/java/org/apache/kafka/common/PartitionInfo.java @@ -19,6 +19,7 @@ /** * Information about a topic-partition. */ +// TODO: rename this class to MetadataPartitionInfo public class PartitionInfo { private final String topic; @@ -26,13 +27,20 @@ public class PartitionInfo { private final Node leader; private final Node[] replicas; private final Node[] inSyncReplicas; + private final Node[] offlineReplicas; + // Used only by tests public PartitionInfo(String topic, int partition, Node leader, Node[] replicas, Node[] inSyncReplicas) { + this(topic, partition, leader, replicas, inSyncReplicas, new Node[0]); + } + + public PartitionInfo(String topic, int partition, Node leader, Node[] replicas, Node[] inSyncReplicas, Node[] offlineReplicas) { this.topic = topic; this.partition = partition; this.leader = leader; this.replicas = replicas; this.inSyncReplicas = inSyncReplicas; + this.offlineReplicas = offlineReplicas; } /** @@ -71,14 +79,22 @@ public Node[] inSyncReplicas() { return inSyncReplicas; } + /** + * The subset of the replicas that are offline + */ + public Node[] offlineReplicas() { + return offlineReplicas; + } + @Override public String toString() { - return String.format("Partition(topic = %s, partition = %d, leader = %s, replicas = %s, isr = %s)", + return String.format("Partition(topic = %s, partition = %d, leader = %s, replicas = %s, isr = %s, offlineReplicas = %s)", topic, partition, leader == null ? "none" : leader.idString(), formatNodeIds(replicas), - formatNodeIds(inSyncReplicas)); + formatNodeIds(inSyncReplicas), + formatNodeIds(offlineReplicas)); } /* Extract the node ids from each item in the array and format for display */ diff --git a/clients/src/main/java/org/apache/kafka/common/errors/KafkaStorageException.java b/clients/src/main/java/org/apache/kafka/common/errors/KafkaStorageException.java new file mode 100644 index 000000000000..a5e5e5432c0f --- /dev/null +++ b/clients/src/main/java/org/apache/kafka/common/errors/KafkaStorageException.java @@ -0,0 +1,33 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.kafka.common.errors; + +/** + * Miscellaneous disk-related IOException occurred when handling a request. + */ +public class KafkaStorageException extends ApiException { + + private static final long serialVersionUID = 1L; + + public KafkaStorageException() { + super(); + } + + public KafkaStorageException(String message) { + super(message); + } +} 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 ae8d16117afd..4ff68ffd8af2 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 @@ -44,6 +44,7 @@ import org.apache.kafka.common.errors.InvalidTopicException; import org.apache.kafka.common.errors.InvalidTxnStateException; import org.apache.kafka.common.errors.InvalidTxnTimeoutException; +import org.apache.kafka.common.errors.KafkaStorageException; import org.apache.kafka.common.errors.LeaderNotAvailableException; import org.apache.kafka.common.errors.NetworkException; import org.apache.kafka.common.errors.NotControllerException; @@ -495,7 +496,14 @@ public ApiException build(String message) { public ApiException build(String message) { return new OperationNotAttemptedException(message); } - }); + }), + KAFKA_STORAGE_ERROR(56, "Disk error when trying to access log file on the disk.", + new ApiExceptionBuilder() { + @Override + public ApiException build(String message) { + return new KafkaStorageException(message); + } + }); private interface ApiExceptionBuilder { ApiException build(String message); diff --git a/clients/src/main/java/org/apache/kafka/common/protocol/Protocol.java b/clients/src/main/java/org/apache/kafka/common/protocol/Protocol.java index 383332b93f00..967cf083b77f 100644 --- a/clients/src/main/java/org/apache/kafka/common/protocol/Protocol.java +++ b/clients/src/main/java/org/apache/kafka/common/protocol/Protocol.java @@ -78,6 +78,9 @@ public class Protocol { "topics that don't exist will be created by the broker. " + "Otherwise, no topics will be created by the broker.")); + /* The v5 metadata request is the same as v4. An additional field for offline_replicas has been added to the v5 metadata response */ + public static final Schema METADATA_REQUEST_V5 = METADATA_REQUEST_V4; + public static final Schema METADATA_BROKER_V0 = new Schema(new Field("node_id", INT32, "The broker id."), new Field("host", STRING, "The hostname of the broker."), new Field("port", INT32, @@ -121,12 +124,38 @@ public class Protocol { public static final Schema PARTITION_METADATA_V1 = PARTITION_METADATA_V0; + public static final Schema PARTITION_METADATA_V2 = new Schema(new Field("partition_error_code", + INT16, + "The error code for the partition, if any."), + new Field("partition_id", + INT32, + "The id of the partition."), + new Field("leader", + INT32, + "The id of the broker acting as leader for this partition."), + new Field("replicas", + new ArrayOf(INT32), + "The set of all nodes that host this partition."), + new Field("isr", + new ArrayOf(INT32), + "The set of nodes that are in sync with the leader for this partition."), + new Field("offline_replicas", + new ArrayOf(INT32), + "The set of offline replicas of this partition.")); + public static final Schema TOPIC_METADATA_V1 = new Schema(new Field("topic_error_code", INT16, "The error code for the given topic."), new Field("topic", STRING, "The name of the topic"), new Field("is_internal", BOOLEAN, "Indicates if the topic is considered a Kafka internal topic"), new Field("partition_metadata", new ArrayOf(PARTITION_METADATA_V1), - "Metadata for each partition of the topic.")); + "Metadata for each partition of the topic.")); + + public static final Schema TOPIC_METADATA_V2 = new Schema(new Field("topic_error_code", INT16, "The error code for the given topic."), + new Field("topic", STRING, "The name of the topic"), + new Field("is_internal", BOOLEAN, + "Indicates if the topic is considered a Kafka internal topic"), + new Field("partition_metadata", new ArrayOf(PARTITION_METADATA_V2), + "Metadata for each partition of the topic.")); public static final Schema METADATA_RESPONSE_V1 = new Schema(new Field("brokers", new ArrayOf(METADATA_BROKER_V1), "Host and port information for all brokers."), @@ -154,8 +183,18 @@ public class Protocol { public static final Schema METADATA_RESPONSE_V4 = METADATA_RESPONSE_V3; - public static final Schema[] METADATA_REQUEST = {METADATA_REQUEST_V0, METADATA_REQUEST_V1, METADATA_REQUEST_V2, METADATA_REQUEST_V3, METADATA_REQUEST_V4}; - public static final Schema[] METADATA_RESPONSE = {METADATA_RESPONSE_V0, METADATA_RESPONSE_V1, METADATA_RESPONSE_V2, METADATA_RESPONSE_V3, METADATA_RESPONSE_V4}; + public static final Schema METADATA_RESPONSE_V5 = new Schema( + newThrottleTimeField(), + new Field("brokers", new ArrayOf(METADATA_BROKER_V1), + "Host and port information for all brokers."), + new Field("cluster_id", NULLABLE_STRING, + "The cluster id that this broker belongs to."), + new Field("controller_id", INT32, + "The broker id of the controller broker."), + new Field("topic_metadata", new ArrayOf(TOPIC_METADATA_V2))); + + public static final Schema[] METADATA_REQUEST = {METADATA_REQUEST_V0, METADATA_REQUEST_V1, METADATA_REQUEST_V2, METADATA_REQUEST_V3, METADATA_REQUEST_V4, METADATA_REQUEST_V5}; + public static final Schema[] METADATA_RESPONSE = {METADATA_RESPONSE_V0, METADATA_RESPONSE_V1, METADATA_RESPONSE_V2, METADATA_RESPONSE_V3, METADATA_RESPONSE_V4, METADATA_RESPONSE_V5}; /* Produce api */ @@ -1039,17 +1078,36 @@ public class Protocol { new Field("zk_version", INT32, "The ZK version."), new Field("replicas", new ArrayOf(INT32), "The replica ids.")); + public static final Schema LEADER_AND_ISR_REQUEST_PARTITION_STATE_V1 = + new Schema(new Field("topic", STRING, "Topic name."), + new Field("partition", INT32, "Topic partition id."), + new Field("controller_epoch", INT32, "The controller epoch."), + new Field("leader", INT32, "The broker id for the leader."), + new Field("leader_epoch", INT32, "The leader epoch."), + new Field("isr", new ArrayOf(INT32), "The in sync replica ids."), + new Field("zk_version", INT32, "The ZK version."), + new Field("replicas", new ArrayOf(INT32), "The replica ids."), + new Field("is_new", BOOLEAN, "If this is false, the replica will only be created if there is no bad log directory")); + public static final Schema LEADER_AND_ISR_REQUEST_LIVE_LEADER_V0 = new Schema(new Field("id", INT32, "The broker id."), new Field("host", STRING, "The hostname of the broker."), new Field("port", INT32, "The port on which the broker accepts requests.")); + public static final Schema LEADER_AND_ISR_REQUEST_LIVE_LEADER_V1 = LEADER_AND_ISR_REQUEST_LIVE_LEADER_V0; + public static final Schema LEADER_AND_ISR_REQUEST_V0 = new Schema(new Field("controller_id", INT32, "The controller id."), new Field("controller_epoch", INT32, "The controller epoch."), new Field("partition_states", new ArrayOf(LEADER_AND_ISR_REQUEST_PARTITION_STATE_V0)), new Field("live_leaders", new ArrayOf(LEADER_AND_ISR_REQUEST_LIVE_LEADER_V0))); + public static final Schema LEADER_AND_ISR_REQUEST_V1 = new Schema(new Field("controller_id", INT32, "The controller id."), + new Field("controller_epoch", INT32, "The controller epoch."), + new Field("partition_states", + new ArrayOf(LEADER_AND_ISR_REQUEST_PARTITION_STATE_V1)), + new Field("live_leaders", new ArrayOf(LEADER_AND_ISR_REQUEST_LIVE_LEADER_V1))); + public static final Schema LEADER_AND_ISR_RESPONSE_PARTITION_V0 = new Schema(new Field("topic", STRING, "Topic name."), new Field("partition", INT32, "Topic partition id."), new Field("error_code", INT16, "Error code.")); @@ -1058,8 +1116,10 @@ public class Protocol { new Field("partitions", new ArrayOf(LEADER_AND_ISR_RESPONSE_PARTITION_V0))); - public static final Schema[] LEADER_AND_ISR_REQUEST = {LEADER_AND_ISR_REQUEST_V0}; - public static final Schema[] LEADER_AND_ISR_RESPONSE = {LEADER_AND_ISR_RESPONSE_V0}; + public static final Schema LEADER_AND_ISR_RESPONSE_V1 = LEADER_AND_ISR_RESPONSE_V0; + + public static final Schema[] LEADER_AND_ISR_REQUEST = {LEADER_AND_ISR_REQUEST_V0, LEADER_AND_ISR_REQUEST_V1}; + public static final Schema[] LEADER_AND_ISR_RESPONSE = {LEADER_AND_ISR_RESPONSE_V0, LEADER_AND_ISR_RESPONSE_V1}; /* Replica api */ public static final Schema STOP_REPLICA_REQUEST_PARTITION_V0 = new Schema(new Field("topic", STRING, "Topic name."), @@ -1141,6 +1201,17 @@ public class Protocol { public static final Schema UPDATE_METADATA_REQUEST_PARTITION_STATE_V3 = UPDATE_METADATA_REQUEST_PARTITION_STATE_V2; + public static final Schema UPDATE_METADATA_REQUEST_PARTITION_STATE_V4 = + new Schema(new Field("topic", STRING, "Topic name."), + new Field("partition", INT32, "Topic partition id."), + new Field("controller_epoch", INT32, "The controller epoch."), + new Field("leader", INT32, "The broker id for the leader."), + new Field("leader_epoch", INT32, "The leader epoch."), + new Field("isr", new ArrayOf(INT32), "The in sync replica ids."), + new Field("zk_version", INT32, "The ZK version."), + new Field("replicas", new ArrayOf(INT32), "The replica ids."), + new Field("offline_replicas", new ArrayOf(INT32), "The offline replica ids")); + public static final Schema UPDATE_METADATA_REQUEST_END_POINT_V3 = new Schema(new Field("port", INT32, "The port on which the broker accepts requests."), new Field("host", STRING, "The hostname of the broker."), @@ -1158,12 +1229,22 @@ public class Protocol { new Field("partition_states", new ArrayOf(UPDATE_METADATA_REQUEST_PARTITION_STATE_V3)), new Field("live_brokers", new ArrayOf(UPDATE_METADATA_REQUEST_BROKER_V3))); + public static final Schema UPDATE_METADATA_REQUEST_BROKER_V4 = UPDATE_METADATA_REQUEST_BROKER_V3; + + public static final Schema UPDATE_METADATA_REQUEST_V4 = + new Schema(new Field("controller_id", INT32, "The controller id."), + new Field("controller_epoch", INT32, "The controller epoch."), + new Field("partition_states", new ArrayOf(UPDATE_METADATA_REQUEST_PARTITION_STATE_V4)), + new Field("live_brokers", new ArrayOf(UPDATE_METADATA_REQUEST_BROKER_V4))); + public static final Schema UPDATE_METADATA_RESPONSE_V3 = UPDATE_METADATA_RESPONSE_V2; + public static final Schema UPDATE_METADATA_RESPONSE_V4 = UPDATE_METADATA_RESPONSE_V3; + public static final Schema[] UPDATE_METADATA_REQUEST = {UPDATE_METADATA_REQUEST_V0, UPDATE_METADATA_REQUEST_V1, - UPDATE_METADATA_REQUEST_V2, UPDATE_METADATA_REQUEST_V3}; + UPDATE_METADATA_REQUEST_V2, UPDATE_METADATA_REQUEST_V3, UPDATE_METADATA_REQUEST_V4}; public static final Schema[] UPDATE_METADATA_RESPONSE = {UPDATE_METADATA_RESPONSE_V0, UPDATE_METADATA_RESPONSE_V1, - UPDATE_METADATA_RESPONSE_V2, UPDATE_METADATA_RESPONSE_V3}; + UPDATE_METADATA_RESPONSE_V2, UPDATE_METADATA_RESPONSE_V3, UPDATE_METADATA_RESPONSE_V4}; /* SASL handshake api */ public static final Schema SASL_HANDSHAKE_REQUEST_V0 = new Schema( diff --git a/clients/src/main/java/org/apache/kafka/common/record/FileRecords.java b/clients/src/main/java/org/apache/kafka/common/record/FileRecords.java index 35431d81592d..a898634a4542 100644 --- a/clients/src/main/java/org/apache/kafka/common/record/FileRecords.java +++ b/clients/src/main/java/org/apache/kafka/common/record/FileRecords.java @@ -171,6 +171,13 @@ public void close() throws IOException { channel.close(); } + /** + * Close file handlers used by the FileChannel but don't write to disk. This is used when the disk may have failed + */ + public void closeHandlers() throws IOException { + channel.close(); + } + /** * Delete this message set from the filesystem * @return True iff this message set was deleted. diff --git a/clients/src/main/java/org/apache/kafka/common/requests/LeaderAndIsrRequest.java b/clients/src/main/java/org/apache/kafka/common/requests/LeaderAndIsrRequest.java index 1fdb4a2be49d..63ce76d2a4a0 100644 --- a/clients/src/main/java/org/apache/kafka/common/requests/LeaderAndIsrRequest.java +++ b/clients/src/main/java/org/apache/kafka/common/requests/LeaderAndIsrRequest.java @@ -45,6 +45,7 @@ public class LeaderAndIsrRequest extends AbstractRequest { private static final String ISR_KEY_NAME = "isr"; private static final String ZK_VERSION_KEY_NAME = "zk_version"; private static final String REPLICAS_KEY_NAME = "replicas"; + private static final String IS_NEW_KEY_NAME = "is_new"; // live_leaders key names private static final String END_POINT_ID_KEY_NAME = "id"; @@ -121,8 +122,9 @@ public LeaderAndIsrRequest(Struct struct, short version) { List replicas = new ArrayList<>(replicasArray.length); for (Object r : replicasArray) replicas.add((Integer) r); + boolean isNew = partitionStateData.hasField(IS_NEW_KEY_NAME) ? partitionStateData.getBoolean(IS_NEW_KEY_NAME) : false; - PartitionState partitionState = new PartitionState(controllerEpoch, leader, leaderEpoch, isr, zkVersion, replicas); + PartitionState partitionState = new PartitionState(controllerEpoch, leader, leaderEpoch, isr, zkVersion, replicas, isNew); partitionStates.put(new TopicPartition(topic, partition), partitionState); } @@ -162,6 +164,8 @@ protected Struct toStruct() { partitionStateData.set(ISR_KEY_NAME, partitionState.isr.toArray()); partitionStateData.set(ZK_VERSION_KEY_NAME, partitionState.zkVersion); partitionStateData.set(REPLICAS_KEY_NAME, partitionState.replicas.toArray()); + if (partitionStateData.hasField(IS_NEW_KEY_NAME)) + partitionStateData.set(IS_NEW_KEY_NAME, partitionState.isNew); partitionStatesData.add(partitionStateData); } struct.set(PARTITION_STATES_KEY_NAME, partitionStatesData.toArray()); @@ -188,6 +192,7 @@ public AbstractResponse getErrorResponse(int throttleTimeMs, Throwable e) { short versionId = version(); switch (versionId) { case 0: + case 1: return new LeaderAndIsrResponse(Errors.NONE, responses); default: throw new IllegalArgumentException(String.format("Version %d is not valid. Valid versions for %s are 0 to %d", 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 b79876464e0f..678ec296aa90 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 @@ -80,6 +80,7 @@ public class MetadataResponse extends AbstractResponse { private static final String LEADER_KEY_NAME = "leader"; private static final String REPLICAS_KEY_NAME = "replicas"; private static final String ISR_KEY_NAME = "isr"; + private static final String OFFLINE_REPLICAS_KEY_NAME = "offline_replicas"; private final int throttleTimeMs; private final Collection brokers; @@ -149,26 +150,18 @@ public MetadataResponse(Struct struct) { int partition = partitionInfo.getInt(PARTITION_KEY_NAME); int leader = partitionInfo.getInt(LEADER_KEY_NAME); Node leaderNode = leader == -1 ? null : brokers.get(leader); - Object[] replicas = (Object[]) partitionInfo.get(REPLICAS_KEY_NAME); - List replicaNodes = new ArrayList<>(replicas.length); - for (Object replicaNodeId : replicas) { - if (brokers.containsKey(replicaNodeId)) - replicaNodes.add(brokers.get(replicaNodeId)); - else - replicaNodes.add(new Node((int) replicaNodeId, "", -1)); - } + Object[] replicas = (Object[]) partitionInfo.get(REPLICAS_KEY_NAME); + List replicaNodes = convertToNodes(brokers, replicas); Object[] isr = (Object[]) partitionInfo.get(ISR_KEY_NAME); - List isrNodes = new ArrayList<>(isr.length); - for (Object isrNode : isr) { - if (brokers.containsKey(isrNode)) - isrNodes.add(brokers.get(isrNode)); - else - isrNodes.add(new Node((int) isrNode, "", -1)); - } + List isrNodes = convertToNodes(brokers, isr); + + Object[] offlineReplicas = partitionInfo.hasField(OFFLINE_REPLICAS_KEY_NAME) ? + (Object[]) partitionInfo.get(OFFLINE_REPLICAS_KEY_NAME) : new Object[0]; + List offlineNodes = convertToNodes(brokers, offlineReplicas); - partitionMetadata.add(new PartitionMetadata(partitionError, partition, leaderNode, replicaNodes, isrNodes)); + partitionMetadata.add(new PartitionMetadata(partitionError, partition, leaderNode, replicaNodes, isrNodes, offlineNodes)); } topicMetadata.add(new TopicMetadata(topicError, topic, isInternal, partitionMetadata)); @@ -179,6 +172,16 @@ public MetadataResponse(Struct struct) { this.topicMetadata = topicMetadata; } + private List convertToNodes(Map brokers, Object[] brokerIds) { + List nodes = new ArrayList<>(brokerIds.length); + for (Object brokerId : brokerIds) + if (brokers.containsKey(brokerId)) + nodes.add(brokers.get(brokerId)); + else + nodes.add(new Node((int) brokerId, "", -1)); + return nodes; + } + private Node getControllerNode(int controllerId, Collection brokers) { for (Node broker : brokers) { if (broker.id() == controllerId) @@ -256,7 +259,8 @@ public Cluster cluster() { partitionMetadata.partition, partitionMetadata.leader, partitionMetadata.replicas.toArray(new Node[0]), - partitionMetadata.isr.toArray(new Node[0]))); + partitionMetadata.isr.toArray(new Node[0]), + partitionMetadata.offlineReplicas.toArray(new Node[0]))); } } @@ -334,23 +338,27 @@ public List partitionMetadata() { } + // TODO: rename this class to MetadataResponsePartitionState public static class PartitionMetadata { private final Errors error; private final int partition; private final Node leader; private final List replicas; private final List isr; + private final List offlineReplicas; public PartitionMetadata(Errors error, int partition, Node leader, List replicas, - List isr) { + List isr, + List offlineReplicas) { this.error = error; this.partition = partition; this.leader = leader; this.replicas = replicas; this.isr = isr; + this.offlineReplicas = offlineReplicas; } public Errors error() { @@ -373,6 +381,10 @@ public List isr() { return isr; } + public List offlineReplicas() { + return offlineReplicas; + } + @Override public String toString() { return "(type=PartitionMetadata," + @@ -433,6 +445,12 @@ protected Struct toStruct(short version) { for (Node node : partitionMetadata.isr) isr.add(node.id()); partitionData.set(ISR_KEY_NAME, isr.toArray()); + if (partitionData.hasField(OFFLINE_REPLICAS_KEY_NAME)) { + ArrayList offlineReplicas = new ArrayList<>(partitionMetadata.offlineReplicas.size()); + for (Node node : partitionMetadata.offlineReplicas) + offlineReplicas.add(node.id()); + partitionData.set(OFFLINE_REPLICAS_KEY_NAME, offlineReplicas.toArray()); + } partitionMetadataArray.add(partitionData); } diff --git a/clients/src/main/java/org/apache/kafka/common/requests/PartitionState.java b/clients/src/main/java/org/apache/kafka/common/requests/PartitionState.java index 394a60f6010d..41bd2e123487 100644 --- a/clients/src/main/java/org/apache/kafka/common/requests/PartitionState.java +++ b/clients/src/main/java/org/apache/kafka/common/requests/PartitionState.java @@ -20,6 +20,7 @@ import java.util.List; +// TODO: rename this class to LeaderAndIsrPartitionInfo public class PartitionState { public final int controllerEpoch; public final int leader; @@ -27,14 +28,16 @@ public class PartitionState { public final List isr; public final int zkVersion; public final List replicas; + public final boolean isNew; - public PartitionState(int controllerEpoch, int leader, int leaderEpoch, List isr, int zkVersion, List replicas) { + public PartitionState(int controllerEpoch, int leader, int leaderEpoch, List isr, int zkVersion, List replicas, boolean isNew) { this.controllerEpoch = controllerEpoch; this.leader = leader; this.leaderEpoch = leaderEpoch; this.isr = isr; this.zkVersion = zkVersion; this.replicas = replicas; + this.isNew = isNew; } @Override @@ -44,6 +47,7 @@ public String toString() { ", leaderEpoch=" + leaderEpoch + ", isr=" + Utils.join(isr, ",") + ", zkVersion=" + zkVersion + - ", replicas=" + Utils.join(replicas, ",") + ")"; + ", replicas=" + Utils.join(replicas, ",") + + ", isNew=" + isNew + ")"; } } diff --git a/clients/src/main/java/org/apache/kafka/common/requests/UpdateMetadataRequest.java b/clients/src/main/java/org/apache/kafka/common/requests/UpdateMetadataRequest.java index 8f9b592c431d..197d35083f0b 100644 --- a/clients/src/main/java/org/apache/kafka/common/requests/UpdateMetadataRequest.java +++ b/clients/src/main/java/org/apache/kafka/common/requests/UpdateMetadataRequest.java @@ -27,6 +27,7 @@ import java.nio.ByteBuffer; import java.util.ArrayList; +import java.util.Arrays; import java.util.HashMap; import java.util.HashSet; import java.util.List; @@ -37,11 +38,11 @@ public class UpdateMetadataRequest extends AbstractRequest { public static class Builder extends AbstractRequest.Builder { private final int controllerId; private final int controllerEpoch; - private final Map partitionStates; + private final Map partitionStates; private final Set liveBrokers; public Builder(short version, int controllerId, int controllerEpoch, - Map partitionStates, Set liveBrokers) { + Map partitionStates, Set liveBrokers) { super(ApiKeys.UPDATE_METADATA_KEY, version); this.controllerId = controllerId; this.controllerEpoch = controllerEpoch; @@ -74,6 +75,43 @@ public String toString() { } } + // TODO: rename this class to UpdateMetadataPartitionState and probably move it to its own class + public static final class UpdateMetadataRequestPartitionState { + public final int controllerEpoch; + public final int leader; + public final int leaderEpoch; + public final List isr; + public final int zkVersion; + public final List replicas; + public final List offlineReplicas; + + public UpdateMetadataRequestPartitionState(int controllerEpoch, + int leader, int leaderEpoch, + List isr, + int zkVersion, + List replicas, + List offlineReplicas) { + this.controllerEpoch = controllerEpoch; + this.leader = leader; + this.leaderEpoch = leaderEpoch; + this.isr = isr; + this.zkVersion = zkVersion; + this.replicas = replicas; + this.offlineReplicas = offlineReplicas; + } + + @Override + public String toString() { + return "UpdateMetadataRequestPartitionState(controllerEpoch=" + controllerEpoch + + ", leader=" + leader + + ", leaderEpoch=" + leaderEpoch + + ", isr=" + Arrays.toString(isr.toArray()) + + ", zkVersion=" + zkVersion + + ", replicas=" + Arrays.toString(replicas.toArray()) + + ", offlineReplicas=" + Arrays.toString(replicas.toArray()) + ")"; + } + } + public static final class Broker { public final int id; public final List endPoints; @@ -121,7 +159,7 @@ public String toString() { private static final String PARTITION_STATES_KEY_NAME = "partition_states"; private static final String LIVE_BROKERS_KEY_NAME = "live_brokers"; - // PartitionState key names + // UpdateMetadataRequestPartitionState key names private static final String TOPIC_KEY_NAME = "topic"; private static final String PARTITION_KEY_NAME = "partition"; private static final String LEADER_KEY_NAME = "leader"; @@ -129,6 +167,7 @@ public String toString() { private static final String ISR_KEY_NAME = "isr"; private static final String ZK_VERSION_KEY_NAME = "zk_version"; private static final String REPLICAS_KEY_NAME = "replicas"; + private static final String OFFLINE_REPLICAS_KEY_NAME = "offline_replicas"; // Broker key names private static final String BROKER_ID_KEY_NAME = "id"; @@ -143,11 +182,11 @@ public String toString() { private final int controllerId; private final int controllerEpoch; - private final Map partitionStates; + private final Map partitionStates; private final Set liveBrokers; - private UpdateMetadataRequest(short version, int controllerId, int controllerEpoch, Map partitionStates, Set liveBrokers) { + private UpdateMetadataRequest(short version, int controllerId, int controllerEpoch, + Map partitionStates, Set liveBrokers) { super(version); this.controllerId = controllerId; this.controllerEpoch = controllerEpoch; @@ -157,7 +196,7 @@ private UpdateMetadataRequest(short version, int controllerId, int controllerEpo public UpdateMetadataRequest(Struct struct, short versionId) { super(versionId); - Map partitionStates = new HashMap<>(); + Map partitionStates = new HashMap<>(); for (Object partitionStateDataObj : struct.getArray(PARTITION_STATES_KEY_NAME)) { Struct partitionStateData = (Struct) partitionStateDataObj; String topic = partitionStateData.getString(TOPIC_KEY_NAME); @@ -178,9 +217,15 @@ public UpdateMetadataRequest(Struct struct, short versionId) { for (Object r : replicasArray) replicas.add((Integer) r); - PartitionState partitionState = new PartitionState(controllerEpoch, leader, leaderEpoch, isr, zkVersion, replicas); - partitionStates.put(new TopicPartition(topic, partition), partitionState); + Object[] offlineReplicasArray = partitionStateData.hasField(OFFLINE_REPLICAS_KEY_NAME) ? + partitionStateData.getArray(OFFLINE_REPLICAS_KEY_NAME) : new Object[0]; + List offlineReplicas = new ArrayList<>(offlineReplicasArray.length); + for (Object r : offlineReplicasArray) + offlineReplicas.add((Integer) r); + UpdateMetadataRequestPartitionState partitionState = + new UpdateMetadataRequestPartitionState(controllerEpoch, leader, leaderEpoch, isr, zkVersion, replicas, offlineReplicas); + partitionStates.put(new TopicPartition(topic, partition), partitionState); } Set liveBrokers = new HashSet<>(); @@ -233,18 +278,20 @@ protected Struct toStruct() { struct.set(CONTROLLER_EPOCH_KEY_NAME, controllerEpoch); List partitionStatesData = new ArrayList<>(partitionStates.size()); - for (Map.Entry entry : partitionStates.entrySet()) { + for (Map.Entry entry : partitionStates.entrySet()) { Struct partitionStateData = struct.instance(PARTITION_STATES_KEY_NAME); TopicPartition topicPartition = entry.getKey(); partitionStateData.set(TOPIC_KEY_NAME, topicPartition.topic()); partitionStateData.set(PARTITION_KEY_NAME, topicPartition.partition()); - PartitionState partitionState = entry.getValue(); + UpdateMetadataRequestPartitionState partitionState = entry.getValue(); partitionStateData.set(CONTROLLER_EPOCH_KEY_NAME, partitionState.controllerEpoch); partitionStateData.set(LEADER_KEY_NAME, partitionState.leader); partitionStateData.set(LEADER_EPOCH_KEY_NAME, partitionState.leaderEpoch); partitionStateData.set(ISR_KEY_NAME, partitionState.isr.toArray()); partitionStateData.set(ZK_VERSION_KEY_NAME, partitionState.zkVersion); partitionStateData.set(REPLICAS_KEY_NAME, partitionState.replicas.toArray()); + if (partitionStateData.hasField(OFFLINE_REPLICAS_KEY_NAME)) + partitionStateData.set(OFFLINE_REPLICAS_KEY_NAME, partitionState.offlineReplicas.toArray()); partitionStatesData.add(partitionStateData); } struct.set(PARTITION_STATES_KEY_NAME, partitionStatesData.toArray()); @@ -301,7 +348,7 @@ public int controllerEpoch() { return controllerEpoch; } - public Map partitionStates() { + public Map partitionStates() { return partitionStates; } diff --git a/clients/src/test/java/org/apache/kafka/clients/consumer/internals/FetcherTest.java b/clients/src/test/java/org/apache/kafka/clients/consumer/internals/FetcherTest.java index 0801979259d6..9966016b622e 100644 --- a/clients/src/test/java/org/apache/kafka/clients/consumer/internals/FetcherTest.java +++ b/clients/src/test/java/org/apache/kafka/clients/consumer/internals/FetcherTest.java @@ -1959,7 +1959,8 @@ private MetadataResponse newMetadataResponse(String topic, Errors error) { partitionInfo.partition(), partitionInfo.leader(), Arrays.asList(partitionInfo.replicas()), - Arrays.asList(partitionInfo.inSyncReplicas()))); + Arrays.asList(partitionInfo.inSyncReplicas()), + Arrays.asList(partitionInfo.offlineReplicas()))); } } diff --git a/clients/src/test/java/org/apache/kafka/common/PartitionInfoTest.java b/clients/src/test/java/org/apache/kafka/common/PartitionInfoTest.java index 7836023fa68d..7511d64c4414 100644 --- a/clients/src/test/java/org/apache/kafka/common/PartitionInfoTest.java +++ b/clients/src/test/java/org/apache/kafka/common/PartitionInfoTest.java @@ -20,20 +20,21 @@ import org.junit.Test; public class PartitionInfoTest { - + @Test public void testToString() { String topic = "sample"; int partition = 0; Node leader = new Node(0, "localhost", 9092); Node r1 = new Node(1, "localhost", 9093); - Node r2 = new Node(2, "localhost", 9094); + Node r2 = new Node(2, "localhost", 9094); Node[] replicas = new Node[] {leader, r1, r2}; - Node[] inSyncReplicas = new Node[] {leader, r1, r2}; - PartitionInfo partitionInfo = new PartitionInfo(topic, partition, leader, replicas, inSyncReplicas); - - String expected = String.format("Partition(topic = %s, partition = %d, leader = %s, replicas = %s, isr = %s)", - topic, partition, leader.idString(), "[0,1,2]", "[0,1,2]"); + Node[] inSyncReplicas = new Node[] {leader, r1}; + Node[] offlineReplicas = new Node[] {r2}; + PartitionInfo partitionInfo = new PartitionInfo(topic, partition, leader, replicas, inSyncReplicas, offlineReplicas); + + String expected = String.format("Partition(topic = %s, partition = %d, leader = %s, replicas = %s, isr = %s, offlineReplicas = %s)", + topic, partition, leader.idString(), "[0,1,2]", "[0,1]", "[2]"); Assert.assertEquals(expected, partitionInfo.toString()); } diff --git a/clients/src/test/java/org/apache/kafka/common/requests/RequestResponseTest.java b/clients/src/test/java/org/apache/kafka/common/requests/RequestResponseTest.java index a3c277f247c3..f113e8b2e3e9 100644 --- a/clients/src/test/java/org/apache/kafka/common/requests/RequestResponseTest.java +++ b/clients/src/test/java/org/apache/kafka/common/requests/RequestResponseTest.java @@ -723,10 +723,11 @@ private MetadataResponse createMetadataResponse() { Node node = new Node(1, "host1", 1001); List replicas = asList(node); List isr = asList(node); + List offlineReplicas = asList(); List allTopicMetadata = new ArrayList<>(); allTopicMetadata.add(new MetadataResponse.TopicMetadata(Errors.NONE, "__consumer_offsets", true, - asList(new MetadataResponse.PartitionMetadata(Errors.NONE, 1, node, replicas, isr)))); + asList(new MetadataResponse.PartitionMetadata(Errors.NONE, 1, node, replicas, isr, offlineReplicas)))); allTopicMetadata.add(new MetadataResponse.TopicMetadata(Errors.LEADER_NOT_AVAILABLE, "topic2", false, Collections.emptyList())); @@ -807,11 +808,11 @@ private LeaderAndIsrRequest createLeaderAndIsrRequest() { List isr = asList(1, 2); List replicas = asList(1, 2, 3, 4); partitionStates.put(new TopicPartition("topic5", 105), - new PartitionState(0, 2, 1, new ArrayList<>(isr), 2, replicas)); + new PartitionState(0, 2, 1, new ArrayList<>(isr), 2, replicas, false)); partitionStates.put(new TopicPartition("topic5", 1), - new PartitionState(1, 1, 1, new ArrayList<>(isr), 2, replicas)); + new PartitionState(1, 1, 1, new ArrayList<>(isr), 2, replicas, false)); partitionStates.put(new TopicPartition("topic20", 1), - new PartitionState(1, 0, 1, new ArrayList<>(isr), 2, replicas)); + new PartitionState(1, 0, 1, new ArrayList<>(isr), 2, replicas, false)); Set leaders = Utils.mkSet( new Node(0, "test0", 1223), @@ -828,15 +829,16 @@ private LeaderAndIsrResponse createLeaderAndIsrResponse() { } private UpdateMetadataRequest createUpdateMetadataRequest(int version, String rack) { - Map partitionStates = new HashMap<>(); + Map partitionStates = new HashMap<>(); List isr = asList(1, 2); List replicas = asList(1, 2, 3, 4); + List offlineReplicas = asList(); partitionStates.put(new TopicPartition("topic5", 105), - new PartitionState(0, 2, 1, new ArrayList<>(isr), 2, replicas)); + new UpdateMetadataRequest.UpdateMetadataRequestPartitionState(0, 2, 1, isr, 2, replicas, offlineReplicas)); partitionStates.put(new TopicPartition("topic5", 1), - new PartitionState(1, 1, 1, new ArrayList<>(isr), 2, replicas)); + new UpdateMetadataRequest.UpdateMetadataRequestPartitionState(1, 1, 1, isr, 2, replicas, offlineReplicas)); partitionStates.put(new TopicPartition("topic20", 1), - new PartitionState(1, 0, 1, new ArrayList<>(isr), 2, replicas)); + new UpdateMetadataRequest.UpdateMetadataRequestPartitionState(1, 0, 1, isr, 2, replicas, offlineReplicas)); SecurityProtocol plaintext = SecurityProtocol.PLAINTEXT; List endPoints1 = new ArrayList<>(); diff --git a/core/src/main/scala/kafka/api/LeaderAndIsr.scala b/core/src/main/scala/kafka/api/LeaderAndIsr.scala index 474d7a003305..4933f7d3050b 100644 --- a/core/src/main/scala/kafka/api/LeaderAndIsr.scala +++ b/core/src/main/scala/kafka/api/LeaderAndIsr.scala @@ -50,13 +50,28 @@ case class LeaderAndIsr(leader: Int, } } -case class PartitionStateInfo(leaderIsrAndControllerEpoch: LeaderIsrAndControllerEpoch, allReplicas: Seq[Int]) { +case class LeaderAndIsrPartitionState(leaderIsrAndControllerEpoch: LeaderIsrAndControllerEpoch, allReplicas: Seq[Int], isNew: Boolean) { override def toString: String = { val partitionStateInfo = new StringBuilder partitionStateInfo.append("(LeaderAndIsrInfo:" + leaderIsrAndControllerEpoch.toString) partitionStateInfo.append(",ReplicationFactor:" + allReplicas.size + ")") partitionStateInfo.append(",AllReplicas:" + allReplicas.mkString(",") + ")") + partitionStateInfo.append(",isNew:" + isNew + ")") + partitionStateInfo.toString() + } +} + +case class MetadataPartitionState(leaderIsrAndControllerEpoch: LeaderIsrAndControllerEpoch, allReplicas: Seq[Int], offlineReplicas: Seq[Int]) { + + def replicationFactor = allReplicas.size + + override def toString: String = { + val partitionStateInfo = new StringBuilder + partitionStateInfo.append("(LeaderAndIsrInfo:" + leaderIsrAndControllerEpoch.toString) + partitionStateInfo.append(",ReplicationFactor:" + replicationFactor + ")") + partitionStateInfo.append(",AllReplicas:" + allReplicas.mkString(",") + ")") + partitionStateInfo.append(",OfflineReplicas:" + offlineReplicas.mkString(",") + ")") partitionStateInfo.toString() } } diff --git a/core/src/main/scala/kafka/cluster/Partition.scala b/core/src/main/scala/kafka/cluster/Partition.scala index ebf314049c20..fa12184402c9 100755 --- a/core/src/main/scala/kafka/cluster/Partition.scala +++ b/core/src/main/scala/kafka/cluster/Partition.scala @@ -114,13 +114,13 @@ class Partition(val topic: String, def isUnderReplicated: Boolean = isLeaderReplicaLocal && inSyncReplicas.size < assignedReplicas.size - def getOrCreateReplica(replicaId: Int = localBrokerId): Replica = { + def getOrCreateReplica(replicaId: Int = localBrokerId, isNew: Boolean = false): Replica = { assignedReplicaMap.getAndMaybePut(replicaId, { if (isReplicaLocal(replicaId)) { val config = LogConfig.fromProps(logManager.defaultConfig.originals, AdminUtils.fetchEntityConfig(zkUtils, ConfigType.Topic, topic)) - val log = logManager.createLog(topicPartition, config) - val checkpoint = replicaManager.highWatermarkCheckpoints(log.dir.getParentFile.getAbsolutePath) + val log = logManager.createLog(topicPartition, config, isNew) + val checkpoint = replicaManager.highWatermarkCheckpoints(log.dir.getParent) val offsetMap = checkpoint.read if (!offsetMap.contains(topicPartition)) info(s"No checkpointed highwatermark is found for partition $topicPartition") @@ -151,14 +151,8 @@ class Partition(val topic: String, assignedReplicaMap.clear() inSyncReplicas = Set.empty[Replica] leaderReplicaIdOpt = None - try { - logManager.asyncDelete(topicPartition) - removePartitionMetrics() - } catch { - case e: IOException => - fatal(s"Error deleting the log for partition $topicPartition", e) - Exit.halt(1) - } + logManager.asyncDelete(topicPartition) + removePartitionMetrics() } } @@ -176,7 +170,7 @@ class Partition(val topic: String, // to maintain the decision maker controller's epoch in the zookeeper path controllerEpoch = partitionStateInfo.controllerEpoch // add replicas that are new - val newInSyncReplicas = partitionStateInfo.isr.asScala.map(r => getOrCreateReplica(r)).toSet + val newInSyncReplicas = partitionStateInfo.isr.asScala.map(r => getOrCreateReplica(r, partitionStateInfo.isNew)).toSet // remove assigned replicas that have been removed by the controller (assignedReplicas.map(_.brokerId) -- allReplicas).foreach(removeReplica) inSyncReplicas = newInSyncReplicas @@ -230,7 +224,7 @@ class Partition(val topic: String, // to maintain the decision maker controller's epoch in the zookeeper path controllerEpoch = partitionStateInfo.controllerEpoch // add replicas that are new - allReplicas.foreach(r => getOrCreateReplica(r)) + allReplicas.foreach(r => getOrCreateReplica(r, partitionStateInfo.isNew)) // remove assigned replicas that have been removed by the controller (assignedReplicas.map(_.brokerId) -- allReplicas).foreach(removeReplica) inSyncReplicas = Set.empty[Replica] @@ -557,7 +551,7 @@ class Partition(val topic: String, /** * remove deleted log metrics */ - private def removePartitionMetrics() { + def removePartitionMetrics() { removeMetric("UnderReplicated", tags) removeMetric("InSyncReplicasCount", tags) removeMetric("ReplicasCount", tags) diff --git a/core/src/main/scala/kafka/controller/ControllerChannelManager.scala b/core/src/main/scala/kafka/controller/ControllerChannelManager.scala index 369da054db7e..27927f5d24d4 100755 --- a/core/src/main/scala/kafka/controller/ControllerChannelManager.scala +++ b/core/src/main/scala/kafka/controller/ControllerChannelManager.scala @@ -30,7 +30,7 @@ import org.apache.kafka.clients._ import org.apache.kafka.common.metrics.Metrics import org.apache.kafka.common.network._ import org.apache.kafka.common.protocol.{ApiKeys, SecurityProtocol} -import org.apache.kafka.common.requests.UpdateMetadataRequest.EndPoint +import org.apache.kafka.common.requests.UpdateMetadataRequest.{EndPoint, UpdateMetadataRequestPartitionState} import org.apache.kafka.common.requests.{UpdateMetadataRequest, _} import org.apache.kafka.common.security.JaasContext import org.apache.kafka.common.utils.Time @@ -278,13 +278,13 @@ class RequestSendThread(val controllerId: Int, } -class ControllerBrokerRequestBatch(controller: KafkaController) extends Logging { +class ControllerBrokerRequestBatch(controller: KafkaController, eventManager: ControllerEventManager) extends Logging { val controllerContext = controller.controllerContext val controllerId: Int = controller.config.brokerId - val leaderAndIsrRequestMap = mutable.Map.empty[Int, mutable.Map[TopicPartition, PartitionStateInfo]] + val leaderAndIsrRequestMap = mutable.Map.empty[Int, mutable.Map[TopicPartition, LeaderAndIsrPartitionState]] val stopReplicaRequestMap = mutable.Map.empty[Int, Seq[StopReplicaRequestInfo]] val updateMetadataRequestBrokerSet = mutable.Set.empty[Int] - val updateMetadataRequestPartitionInfoMap = mutable.Map.empty[TopicPartition, PartitionStateInfo] + val updateMetadataRequestPartitionInfoMap = mutable.Map.empty[TopicPartition, MetadataPartitionState] private val stateChangeLogger = KafkaController.stateChangeLogger def newBatch() { @@ -310,12 +310,14 @@ class ControllerBrokerRequestBatch(controller: KafkaController) extends Logging def addLeaderAndIsrRequestForBrokers(brokerIds: Seq[Int], topic: String, partition: Int, leaderIsrAndControllerEpoch: LeaderIsrAndControllerEpoch, - replicas: Seq[Int], callback: AbstractResponse => Unit = null) { + replicas: Seq[Int], callback: AbstractResponse => Unit = null, + isNew: Boolean = false) { val topicPartition = new TopicPartition(topic, partition) brokerIds.filter(_ >= 0).foreach { brokerId => val result = leaderAndIsrRequestMap.getOrElseUpdate(brokerId, mutable.Map.empty) - result.put(topicPartition, PartitionStateInfo(leaderIsrAndControllerEpoch, replicas)) + val alreadyNew = result.get(topicPartition).map(_.isNew).getOrElse(false) + result.put(topicPartition, LeaderAndIsrPartitionState(leaderIsrAndControllerEpoch, replicas, isNew || alreadyNew)) } addUpdateMetadataRequestForBrokers(controllerContext.liveOrShuttingDownBrokerIds.toSeq, @@ -345,7 +347,7 @@ class ControllerBrokerRequestBatch(controller: KafkaController) extends Logging leaderIsrAndControllerEpochOpt match { case Some(l @ LeaderIsrAndControllerEpoch(leaderAndIsr, controllerEpoch)) => val replicas = controllerContext.partitionReplicaAssignment(partition) - + val offlineReplicas = replicas.filter(!controllerContext.isReplicaOnline(_, partition)) val leaderIsrAndControllerEpoch = if (beingDeleted) { val leaderDuringDelete = LeaderAndIsr.duringDelete(leaderAndIsr.isr) LeaderIsrAndControllerEpoch(leaderDuringDelete, controllerEpoch) @@ -353,7 +355,7 @@ class ControllerBrokerRequestBatch(controller: KafkaController) extends Logging l } - val partitionStateInfo = PartitionStateInfo(leaderIsrAndControllerEpoch, replicas) + val partitionStateInfo = MetadataPartitionState(leaderIsrAndControllerEpoch, replicas, offlineReplicas) updateMetadataRequestPartitionInfoMap.put(new TopicPartition(partition.topic, partition.partition), partitionStateInfo) case None => @@ -379,8 +381,8 @@ class ControllerBrokerRequestBatch(controller: KafkaController) extends Logging def sendRequestsToBrokers(controllerEpoch: Int) { try { - leaderAndIsrRequestMap.foreach { case (broker, partitionStateInfos) => - partitionStateInfos.foreach { case (topicPartition, state) => + leaderAndIsrRequestMap.foreach { case (broker, leaderAndIsrPartitionStates) => + leaderAndIsrPartitionStates.foreach { case (topicPartition, state) => val typeOfRequest = if (broker == state.leaderIsrAndControllerEpoch.leaderAndIsr.leader) "become-leader" else "become-follower" @@ -389,20 +391,21 @@ class ControllerBrokerRequestBatch(controller: KafkaController) extends Logging state.leaderIsrAndControllerEpoch, broker, topicPartition.topic, topicPartition.partition)) } - val leaderIds = partitionStateInfos.map(_._2.leaderIsrAndControllerEpoch.leaderAndIsr.leader).toSet + val leaderIds = leaderAndIsrPartitionStates.map(_._2.leaderIsrAndControllerEpoch.leaderAndIsr.leader).toSet val leaders = controllerContext.liveOrShuttingDownBrokers.filter(b => leaderIds.contains(b.id)).map { _.getNode(controller.config.interBrokerListenerName) } - val partitionStates = partitionStateInfos.map { case (topicPartition, partitionStateInfo) => - val LeaderIsrAndControllerEpoch(leaderIsr, controllerEpoch) = partitionStateInfo.leaderIsrAndControllerEpoch + val partitionStates = leaderAndIsrPartitionStates.map { case (topicPartition, leaderAndIsrPartitionState) => + val LeaderIsrAndControllerEpoch(leaderIsr, controllerEpoch) = leaderAndIsrPartitionState.leaderIsrAndControllerEpoch val partitionState = new requests.PartitionState(controllerEpoch, leaderIsr.leader, leaderIsr.leaderEpoch, leaderIsr.isr.map(Integer.valueOf).asJava, leaderIsr.zkVersion, - partitionStateInfo.allReplicas.map(Integer.valueOf).asJava) + leaderAndIsrPartitionState.allReplicas.map(Integer.valueOf).asJava, leaderAndIsrPartitionState.isNew) topicPartition -> partitionState } val leaderAndIsrRequest = new LeaderAndIsrRequest.Builder(controllerId, controllerEpoch, partitionStates.asJava, leaders.asJava) - controller.sendRequest(broker, ApiKeys.LEADER_AND_ISR, leaderAndIsrRequest) + controller.sendRequest(broker, ApiKeys.LEADER_AND_ISR, leaderAndIsrRequest, + (r: AbstractResponse) => eventManager.put(controller.LeaderAndIsrResponseReceived(r, broker))) } leaderAndIsrRequestMap.clear() @@ -411,9 +414,9 @@ class ControllerBrokerRequestBatch(controller: KafkaController) extends Logging updateMetadataRequestBrokerSet.toString(), p._1))) val partitionStates = updateMetadataRequestPartitionInfoMap.map { case (topicPartition, partitionStateInfo) => val LeaderIsrAndControllerEpoch(leaderIsr, controllerEpoch) = partitionStateInfo.leaderIsrAndControllerEpoch - val partitionState = new requests.PartitionState(controllerEpoch, leaderIsr.leader, + val partitionState = new UpdateMetadataRequestPartitionState(controllerEpoch, leaderIsr.leader, leaderIsr.leaderEpoch, leaderIsr.isr.map(Integer.valueOf).asJava, leaderIsr.zkVersion, - partitionStateInfo.allReplicas.map(Integer.valueOf).asJava) + partitionStateInfo.allReplicas.map(Integer.valueOf).asJava, partitionStateInfo.offlineReplicas.map(Integer.valueOf).asJava) topicPartition -> partitionState } diff --git a/core/src/main/scala/kafka/controller/ControllerState.scala b/core/src/main/scala/kafka/controller/ControllerState.scala index 2f690bbbdf8c..74029b1b6013 100644 --- a/core/src/main/scala/kafka/controller/ControllerState.scala +++ b/core/src/main/scala/kafka/controller/ControllerState.scala @@ -78,6 +78,15 @@ object ControllerState { def value = 9 } + case object LeaderAndIsrResponseReceived extends ControllerState { + def value = 10 + } + + case object LogDirChange extends ControllerState { + def value = 11 + } + val values: Seq[ControllerState] = Seq(Idle, ControllerChange, BrokerChange, TopicChange, TopicDeletion, - PartitionReassignment, AutoLeaderBalance, ManualLeaderBalance, ControlledShutdown, IsrChange) + PartitionReassignment, AutoLeaderBalance, ManualLeaderBalance, ControlledShutdown, IsrChange, LeaderAndIsrResponseReceived, + LogDirChange) } diff --git a/core/src/main/scala/kafka/controller/KafkaController.scala b/core/src/main/scala/kafka/controller/KafkaController.scala index e7f98e558ecf..9d37917656e7 100644 --- a/core/src/main/scala/kafka/controller/KafkaController.scala +++ b/core/src/main/scala/kafka/controller/KafkaController.scala @@ -18,7 +18,8 @@ package kafka.controller import java.util.concurrent.TimeUnit -import com.yammer.metrics.core.Gauge +import org.apache.kafka.common.requests.LeaderAndIsrResponse +import com.yammer.metrics.core.{Gauge, Meter} import kafka.admin.{AdminUtils, PreferredReplicaLeaderElectionCommand} import kafka.api._ import kafka.cluster.Broker @@ -52,6 +53,8 @@ class ControllerContext(val zkUtils: ZkUtils) { var partitionReplicaAssignment: mutable.Map[TopicAndPartition, Seq[Int]] = mutable.Map.empty var partitionLeadershipInfo: mutable.Map[TopicAndPartition, LeaderIsrAndControllerEpoch] = mutable.Map.empty val partitionsBeingReassigned: mutable.Map[TopicAndPartition, ReassignedPartitionsContext] = new mutable.HashMap + val partitionsUndergoingPreferredReplicaElection: mutable.Set[TopicAndPartition] = new mutable.HashSet + val replicasOnOfflineDisks: mutable.Map[Int, Set[TopicAndPartition]] = mutable.HashMap.empty private var liveBrokersUnderlying: Set[Broker] = Set.empty private var liveBrokerIdsUnderlying: Set[Int] = Set.empty @@ -75,6 +78,14 @@ class ControllerContext(val zkUtils: ZkUtils) { }.toSet } + def isReplicaOnline(brokerId: Int, topicAndPartition: TopicAndPartition, includeShuttingDownBrokers: Boolean = false): Boolean = { + val brokerOnline = { + if (includeShuttingDownBrokers) liveOrShuttingDownBrokerIds.contains(brokerId) + else liveBrokerIds.contains(brokerId) + } + brokerOnline && !replicasOnOfflineDisks.getOrElse(brokerId, Set.empty).contains(topicAndPartition) + } + def replicasOnBrokers(brokerIds: Set[Int]): Set[PartitionAndReplica] = { brokerIds.flatMap { brokerId => partitionReplicaAssignment.collect { @@ -98,7 +109,8 @@ class ControllerContext(val zkUtils: ZkUtils) { partitionReplicaAssignment.keySet.filter(topicAndPartition => topicAndPartition.topic == topic) def allLiveReplicas(): Set[PartitionAndReplica] = { - replicasOnBrokers(liveBrokerIds) + replicasOnBrokers(liveBrokerIds).filter{partitionAndReplica => + isReplicaOnline(partitionAndReplica.replica, TopicAndPartition(partitionAndReplica.topic, partitionAndReplica.partition))} } def replicasForPartition(partitions: collection.Set[TopicAndPartition]): collection.Set[PartitionAndReplica] = { @@ -150,23 +162,25 @@ class KafkaController(val config: KafkaConfig, zkUtils: ZkUtils, time: Time, met this.logIdent = "[Controller " + config.brokerId + "]: " private val stateChangeLogger = KafkaController.stateChangeLogger val controllerContext = new ControllerContext(zkUtils) - val partitionStateMachine = new PartitionStateMachine(this) - val replicaStateMachine = new ReplicaStateMachine(this) + + // visible for testing + private[controller] val eventManager = new ControllerEventManager(controllerContext.stats.rateAndTimeMetrics, + _ => updateMetrics()) + + val partitionStateMachine = new PartitionStateMachine(this, eventManager) + val replicaStateMachine = new ReplicaStateMachine(this, eventManager) // have a separate scheduler for the controller to be able to start and stop independently of the kafka server // visible for testing private[controller] val kafkaScheduler = new KafkaScheduler(1) - // visible for testing - private[controller] val eventManager = new ControllerEventManager(controllerContext.stats.rateAndTimeMetrics, - _ => updateMetrics()) val topicDeletionManager = new TopicDeletionManager(this, eventManager) val offlinePartitionSelector = new OfflinePartitionLeaderSelector(controllerContext, config) private val reassignedPartitionLeaderSelector = new ReassignedPartitionLeaderSelector(controllerContext) private val preferredReplicaPartitionLeaderSelector = new PreferredReplicaPartitionLeaderSelector(controllerContext) private val controlledShutdownPartitionLeaderSelector = new ControlledShutdownLeaderSelector(controllerContext) - private val brokerRequestBatch = new ControllerBrokerRequestBatch(this) + private val brokerRequestBatch = new ControllerBrokerRequestBatch(this, eventManager) private val brokerChangeListener = new BrokerChangeListener(this, eventManager) private val topicChangeListener = new TopicChangeListener(this, eventManager) @@ -175,6 +189,7 @@ class KafkaController(val config: KafkaConfig, zkUtils: ZkUtils, time: Time, met private val partitionReassignmentListener = new PartitionReassignmentListener(this, eventManager) private val preferredReplicaElectionListener = new PreferredReplicaElectionListener(this, eventManager) private val isrChangeNotificationListener = new IsrChangeNotificationListener(this, eventManager) + private val logDirEventNotificationListener = new LogDirEventNotificationListener(this, eventManager) @volatile private var activeControllerId = -1 @volatile private var offlinePartitionCount = 0 @@ -256,6 +271,7 @@ class KafkaController(val config: KafkaConfig, zkUtils: ZkUtils, time: Time, met registerTopicChangeListener() registerTopicDeletionListener() registerBrokerChangeListener() + registerLogDirEventNotificationListener() initializeControllerContext() val (topicsToBeDeleted, topicsIneligibleForDeletion) = fetchTopicDeletionsInProgress() @@ -299,6 +315,7 @@ class KafkaController(val config: KafkaConfig, zkUtils: ZkUtils, time: Time, met deregisterIsrChangeNotificationListener() deregisterPartitionReassignmentListener() deregisterPreferredReplicaElectionListener() + deregisterLogDirEventNotificationListener() // reset topic deletion manager topicDeletionManager.reset() @@ -329,6 +346,12 @@ class KafkaController(val config: KafkaConfig, zkUtils: ZkUtils, time: Time, met */ def isActive: Boolean = activeControllerId == config.brokerId + def onBrokerLogDirFailure(brokerIds: Seq[Int]) { + // send LeaderAndIsrRequest for all live replicas on those brokers to see if they are still online. + val replicasOnBrokers = controllerContext.replicasOnBrokers(brokerIds.toSet) + replicaStateMachine.handleStateChanges(replicasOnBrokers, OnlineReplica) + } + /** * This callback is invoked by the replica state machine's broker change listener, with the list of newly started * brokers as input. It does the following - @@ -345,6 +368,7 @@ class KafkaController(val config: KafkaConfig, zkUtils: ZkUtils, time: Time, met */ def onBrokerStartup(newBrokers: Seq[Int]) { info("New broker startup callback for %s".format(newBrokers.mkString(","))) + newBrokers.foreach(controllerContext.replicasOnOfflineDisks.remove) val newBrokersSet = newBrokers.toSet // send update metadata request to all live and shutting down brokers. Old brokers will get to know of the new // broker via this update. @@ -388,6 +412,7 @@ class KafkaController(val config: KafkaConfig, zkUtils: ZkUtils, time: Time, met */ def onBrokerFailure(deadBrokers: Seq[Int]) { info("Broker failure callback for %s".format(deadBrokers.mkString(","))) + deadBrokers.foreach(controllerContext.replicasOnOfflineDisks.remove) val deadBrokersThatWereShuttingDown = deadBrokers.filter(id => controllerContext.shuttingDownBrokerIds.remove(id)) info("Removed %s from list of shutting down brokers.".format(deadBrokersThatWereShuttingDown)) @@ -724,10 +749,11 @@ class KafkaController(val config: KafkaConfig, zkUtils: ZkUtils, time: Time, met private def fetchTopicDeletionsInProgress(): (Set[String], Set[String]) = { val topicsToBeDeleted = zkUtils.getChildrenParentMayNotExist(ZkUtils.DeleteTopicsPath).toSet - val topicsWithReplicasOnDeadBrokers = controllerContext.partitionReplicaAssignment.filter { case (_, replicas) => - replicas.exists(r => !controllerContext.liveBrokerIds.contains(r)) }.keySet.map(_.topic) + val topicsWithOfflineReplicas = controllerContext.partitionReplicaAssignment.filter { case (partition, replicas) => + replicas.exists(r => !controllerContext.isReplicaOnline(r, partition)) + }.keySet.map(_.topic) val topicsForWhichPartitionReassignmentIsInProgress = controllerContext.partitionsBeingReassigned.keySet.map(_.topic) - val topicsIneligibleForDeletion = topicsWithReplicasOnDeadBrokers | topicsForWhichPartitionReassignmentIsInProgress + val topicsIneligibleForDeletion = topicsWithOfflineReplicas | topicsForWhichPartitionReassignmentIsInProgress info("List of topics to be deleted: %s".format(topicsToBeDeleted.mkString(","))) info("List of topics ineligible for deletion: %s".format(topicsIneligibleForDeletion.mkString(","))) (topicsToBeDeleted, topicsIneligibleForDeletion) @@ -771,7 +797,7 @@ class KafkaController(val config: KafkaConfig, zkUtils: ZkUtils, time: Time, met partitionStateMachine.handleStateChanges(Set(topicAndPartition), OnlinePartition, reassignedPartitionLeaderSelector) } else { // check if the leader is alive or not - if (controllerContext.liveBrokerIds.contains(currentLeader)) { + if (controllerContext.isReplicaOnline(currentLeader, topicAndPartition)) { info("Leader %s for partition %s being reassigned, ".format(currentLeader, topicAndPartition) + "is already in the new list of replicas %s and is alive".format(reassignedReplicas.mkString(","))) // shrink replication factor and update the leader epoch in zookeeper to use on the next LeaderAndIsrRequest @@ -909,6 +935,16 @@ class KafkaController(val config: KafkaConfig, zkUtils: ZkUtils, time: Time, met } } + private def registerLogDirEventNotificationListener() = { + debug("Registering logDirEventNotificationListener") + zkUtils.zkClient.subscribeChildChanges(ZkUtils.LogDirEventNotificationPath, logDirEventNotificationListener) + } + + private def deregisterLogDirEventNotificationListener() = { + debug("De-registering logDirEventNotificationListener") + zkUtils.zkClient.unsubscribeChildChanges(ZkUtils.LogDirEventNotificationPath, logDirEventNotificationListener) + } + private def readControllerEpochFromZookeeper() { // initialize the controller epoch and zk version by reading from zookeeper if(controllerContext.zkUtils.pathExists(ZkUtils.ControllerEpochPath)) { @@ -1119,7 +1155,7 @@ class KafkaController(val config: KafkaConfig, zkUtils: ZkUtils, time: Time, met topicsNotInPreferredReplica.keys.foreach { topicPartition => // do this check only if the broker is live and there are no partitions being reassigned currently // and preferred replica election is not in progress - if (controllerContext.liveBrokerIds.contains(leaderBroker) && + if (controllerContext.isReplicaOnline(leaderBroker, topicPartition) && controllerContext.partitionsBeingReassigned.isEmpty && !topicDeletionManager.isTopicQueuedUpForDeletion(topicPartition.topic) && controllerContext.allTopics.contains(topicPartition.topic)) { @@ -1354,6 +1390,22 @@ class KafkaController(val config: KafkaConfig, zkUtils: ZkUtils, time: Time, met } + case class LogDirEventNotification(sequenceNumbers: Seq[String]) extends ControllerEvent { + + def state = ControllerState.LogDirChange + + override def process(): Unit = { + val zkUtils = controllerContext.zkUtils + try { + val brokerIds = sequenceNumbers.flatMap(DiskUtils.getBrokerIdFromLogDirEvent(zkUtils, _)) + onBrokerLogDirFailure(brokerIds) + } finally { + // delete processed children + sequenceNumbers.map(x => zkUtils.deletePath(ZkUtils.LogDirEventNotificationPath + "/" + x)) + } + } + } + case class PreferredReplicaLeaderElection(partitions: Set[TopicAndPartition]) extends ControllerEvent { def state = ControllerState.ManualLeaderBalance @@ -1449,7 +1501,56 @@ class KafkaController(val config: KafkaConfig, zkUtils: ZkUtils, time: Time, met } } - case class TopicDeletionStopReplicaResult(stopReplicaResponseObj: AbstractResponse, replicaId: Int) extends ControllerEvent { + case class LeaderAndIsrResponseReceived(LeaderAndIsrResponseObj: AbstractResponse, brokerId: Int) extends ControllerEvent { + + def state = ControllerState.LeaderAndIsrResponseReceived + + override def process(): Unit = { + import JavaConverters._ + val leaderAndIsrResponse = LeaderAndIsrResponseObj.asInstanceOf[LeaderAndIsrResponse] + + if (leaderAndIsrResponse.error() != Errors.NONE) { + stateChangeLogger.error(s"Received error in leaderAndIsrResponse $leaderAndIsrResponse from broker $brokerId") + return + } + + val offlineReplicas = leaderAndIsrResponse.responses().asScala.filter(_._2 == Errors.KAFKA_STORAGE_ERROR).keys.map( + tp => TopicAndPartition(tp.topic(), tp.partition())).toSet + val onlineReplicas = leaderAndIsrResponse.responses().asScala.filter(_._2 == Errors.NONE).keys.map( + tp => TopicAndPartition(tp.topic(), tp.partition())).toSet + val previousOfflineReplicas = controllerContext.replicasOnOfflineDisks.getOrElse(brokerId, Set.empty[TopicAndPartition]) + val currentOfflineReplicas = previousOfflineReplicas -- onlineReplicas ++ offlineReplicas + controllerContext.replicasOnOfflineDisks.put(brokerId, currentOfflineReplicas) + val newOfflineReplicas = (currentOfflineReplicas -- previousOfflineReplicas).map(tp => PartitionAndReplica(tp.topic, tp.partition, brokerId)) + stateChangeLogger.info(s"Mark replicas ${currentOfflineReplicas -- previousOfflineReplicas} on broker ${brokerId} as offline") + + if (newOfflineReplicas.nonEmpty) { + val (newOfflineReplicasForDeletion, newOfflineReplicasNotForDeletion) = + newOfflineReplicas.partition(p => topicDeletionManager.isTopicQueuedUpForDeletion(p.topic)) + + // trigger OfflinePartition state change for all partitions whose leader is on the failed disk + val partitionsWithoutLeader = controllerContext.partitionLeadershipInfo.filter(partitionAndLeader => + !controllerContext.isReplicaOnline(partitionAndLeader._2.leaderAndIsr.leader, partitionAndLeader._1) && + !topicDeletionManager.isTopicQueuedUpForDeletion(partitionAndLeader._1.topic)).keySet + partitionStateMachine.handleStateChanges(partitionsWithoutLeader, OfflinePartition) + // trigger OnlinePartition state change for offline or new partitions + partitionStateMachine.triggerOnlinePartitionStateChange() + + // trigger OfflineReplica state change for those newly-discovered offline replicas + replicaStateMachine.handleStateChanges(newOfflineReplicasNotForDeletion, OfflineReplica) + + // fail deletion of topics that affected by the offline replicas + if (newOfflineReplicasForDeletion.nonEmpty) { + // it is required to mark the respective replicas in TopicDeletionFailed state since the replica cannot be + // deleted when its disk is down. This will prevent the replica from being in TopicDeletionStarted state indefinitely + // since topic deletion cannot be retried until at least one replica is in TopicDeletionStarted state + topicDeletionManager.failReplicaDeletion(newOfflineReplicasForDeletion) + } + } + } + } + + case class TopicDeletionStopReplicaResponseReceived(stopReplicaResponseObj: AbstractResponse, replicaId: Int) extends ControllerEvent { def state = ControllerState.TopicDeletion @@ -1609,6 +1710,22 @@ class TopicChangeListener(controller: KafkaController, eventManager: ControllerE } } +/** + * Called when broker notifies controller of disk change + * + * @param controller + */ +class LogDirEventNotificationListener(controller: KafkaController, eventManager: ControllerEventManager) extends IZkChildListener with Logging { + override def handleChildChange(parentPath: String, currentChilds: java.util.List[String]): Unit = { + import JavaConverters._ + eventManager.put(controller.LogDirEventNotification(currentChilds.asScala)) + } +} + +object LogDirEventNotificationListener { + val version: Long = 1L +} + class PartitionModificationsListener(controller: KafkaController, eventManager: ControllerEventManager, topic: String) extends IZkDataListener with Logging { override def handleDataChange(dataPath: String, data: Any): Unit = { eventManager.put(controller.PartitionModifications(topic)) diff --git a/core/src/main/scala/kafka/controller/PartitionLeaderSelector.scala b/core/src/main/scala/kafka/controller/PartitionLeaderSelector.scala index 54bbb8938f9d..a66a9933b70e 100644 --- a/core/src/main/scala/kafka/controller/PartitionLeaderSelector.scala +++ b/core/src/main/scala/kafka/controller/PartitionLeaderSelector.scala @@ -54,8 +54,8 @@ class OfflinePartitionLeaderSelector(controllerContext: ControllerContext, confi def selectLeader(topicAndPartition: TopicAndPartition, currentLeaderAndIsr: LeaderAndIsr): (LeaderAndIsr, Seq[Int]) = { controllerContext.partitionReplicaAssignment.get(topicAndPartition) match { case Some(assignedReplicas) => - val liveAssignedReplicas = assignedReplicas.filter(r => controllerContext.liveBrokerIds.contains(r)) - val liveBrokersInIsr = currentLeaderAndIsr.isr.filter(r => controllerContext.liveBrokerIds.contains(r)) + val liveAssignedReplicas = assignedReplicas.filter(r => controllerContext.isReplicaOnline(r, topicAndPartition)) + val liveBrokersInIsr = currentLeaderAndIsr.isr.filter(r => controllerContext.isReplicaOnline(r, topicAndPartition)) val newLeaderAndIsr = if (liveBrokersInIsr.isEmpty) { // Prior to electing an unclean (i.e. non-ISR) leader, ensure that doing so is not disallowed by the configuration @@ -111,7 +111,7 @@ class ReassignedPartitionLeaderSelector(controllerContext: ControllerContext) ex currentLeaderAndIsr: LeaderAndIsr): (LeaderAndIsr, Seq[Int]) = { val reassignedInSyncReplicas = controllerContext.partitionsBeingReassigned(topicAndPartition).newReplicas val newLeaderOpt = reassignedInSyncReplicas.find { r => - controllerContext.liveBrokerIds.contains(r) && currentLeaderAndIsr.isr.contains(r) + controllerContext.isReplicaOnline(r, topicAndPartition) && currentLeaderAndIsr.isr.contains(r) } newLeaderOpt match { case Some(newLeader) => (currentLeaderAndIsr.newLeader(newLeader), reassignedInSyncReplicas) @@ -174,8 +174,7 @@ class ControlledShutdownLeaderSelector(controllerContext: ControllerContext) ext currentLeaderAndIsr: LeaderAndIsr): (LeaderAndIsr, Seq[Int]) = { val currentIsr = currentLeaderAndIsr.isr val assignedReplicas = controllerContext.partitionReplicaAssignment(topicAndPartition) - val liveOrShuttingDownBrokerIds = controllerContext.liveOrShuttingDownBrokerIds - val liveAssignedReplicas = assignedReplicas.filter(r => liveOrShuttingDownBrokerIds.contains(r)) + val liveAssignedReplicas = assignedReplicas.filter(r => controllerContext.isReplicaOnline(r, topicAndPartition, true)) val newIsr = currentIsr.filter(brokerId => !controllerContext.shuttingDownBrokerIds.contains(brokerId)) liveAssignedReplicas.find(newIsr.contains) match { diff --git a/core/src/main/scala/kafka/controller/PartitionStateMachine.scala b/core/src/main/scala/kafka/controller/PartitionStateMachine.scala index 5751e17de0c0..20501fd204c5 100755 --- a/core/src/main/scala/kafka/controller/PartitionStateMachine.scala +++ b/core/src/main/scala/kafka/controller/PartitionStateMachine.scala @@ -37,12 +37,12 @@ import scala.collection._ * 4. OfflinePartition : If, after successful leader election, the leader for partition dies, then the partition * moves to the OfflinePartition state. Valid previous states are NewPartition/OnlinePartition */ -class PartitionStateMachine(controller: KafkaController) extends Logging { +class PartitionStateMachine(controller: KafkaController, eventManager: ControllerEventManager) extends Logging { private val controllerContext = controller.controllerContext private val controllerId = controller.config.brokerId private val zkUtils = controllerContext.zkUtils private val partitionState: mutable.Map[TopicAndPartition, PartitionState] = mutable.Map.empty - private val brokerRequestBatch = new ControllerBrokerRequestBatch(controller) + private val brokerRequestBatch = new ControllerBrokerRequestBatch(controller, eventManager) private val noOpPartitionLeaderSelector = new NoOpLeaderSelector(controllerContext) private val stateChangeLogger = KafkaController.stateChangeLogger @@ -200,7 +200,7 @@ class PartitionStateMachine(controller: KafkaController) extends Logging { controllerContext.partitionLeadershipInfo.get(topicPartition) match { case Some(currentLeaderIsrAndEpoch) => // else, check if the leader for partition is alive. If yes, it is in Online state, else it is in Offline state - if (controllerContext.liveBrokerIds.contains(currentLeaderIsrAndEpoch.leaderAndIsr.leader)) + if (controllerContext.isReplicaOnline(currentLeaderIsrAndEpoch.leaderAndIsr.leader, topicPartition)) // leader is alive partitionState.put(topicPartition, OnlinePartition) else @@ -227,7 +227,7 @@ class PartitionStateMachine(controller: KafkaController) extends Logging { */ private def initializeLeaderAndIsrForPartition(topicAndPartition: TopicAndPartition) = { val replicaAssignment = controllerContext.partitionReplicaAssignment(topicAndPartition).toList - val liveAssignedReplicas = replicaAssignment.filter(controllerContext.liveBrokerIds.contains) + val liveAssignedReplicas = replicaAssignment.filter(r => controllerContext.isReplicaOnline(r, topicAndPartition)) liveAssignedReplicas.headOption match { case None => val failMsg = s"Controller $controllerId epoch ${controller.epoch} encountered error during state change of " + @@ -259,7 +259,9 @@ class PartitionStateMachine(controller: KafkaController) extends Logging { topicAndPartition.topic, topicAndPartition.partition, leaderIsrAndControllerEpoch, - replicaAssignment + replicaAssignment, + callback = null, + isNew = true ) } catch { case _: ZkNodeExistsException => diff --git a/core/src/main/scala/kafka/controller/ReplicaStateMachine.scala b/core/src/main/scala/kafka/controller/ReplicaStateMachine.scala index 60b99908d563..3b8f94c8f8fc 100755 --- a/core/src/main/scala/kafka/controller/ReplicaStateMachine.scala +++ b/core/src/main/scala/kafka/controller/ReplicaStateMachine.scala @@ -40,12 +40,12 @@ import scala.collection._ * 7. NonExistentReplica: If a replica is deleted successfully, it is moved to this state. Valid previous state is * ReplicaDeletionSuccessful */ -class ReplicaStateMachine(controller: KafkaController) extends Logging { +class ReplicaStateMachine(controller: KafkaController, eventManager: ControllerEventManager) extends Logging { private val controllerContext = controller.controllerContext private val controllerId = controller.config.brokerId private val zkUtils = controllerContext.zkUtils private val replicaState: mutable.Map[PartitionAndReplica, ReplicaState] = mutable.Map.empty - private val brokerRequestBatch = new ControllerBrokerRequestBatch(controller) + private val brokerRequestBatch = new ControllerBrokerRequestBatch(controller, eventManager) private val stateChangeLogger = KafkaController.stateChangeLogger @@ -149,7 +149,7 @@ class ReplicaStateMachine(controller: KafkaController) extends Logging { .format(replicaId, topicAndPartition) + "state as it is being requested to become leader") brokerRequestBatch.addLeaderAndIsrRequestForBrokers(List(replicaId), topic, partition, leaderIsrAndControllerEpoch, - replicaAssignment) + replicaAssignment, callback = null, isNew = true) case None => // new leader request will be sent to this replica when one gets elected } replicaState.put(partitionAndReplica, NewReplica) @@ -283,7 +283,7 @@ class ReplicaStateMachine(controller: KafkaController) extends Logging { val partition = topicPartition.partition assignedReplicas.foreach { replicaId => val partitionAndReplica = PartitionAndReplica(topic, partition, replicaId) - if (controllerContext.liveBrokerIds.contains(replicaId)) + if (controllerContext.isReplicaOnline(replicaId, topicPartition)) replicaState.put(partitionAndReplica, OnlineReplica) else // mark replicas on dead brokers as failed for topic deletion, if they belong to a topic to be deleted. diff --git a/core/src/main/scala/kafka/controller/TopicDeletionManager.scala b/core/src/main/scala/kafka/controller/TopicDeletionManager.scala index e483ac233176..325488eb2022 100755 --- a/core/src/main/scala/kafka/controller/TopicDeletionManager.scala +++ b/core/src/main/scala/kafka/controller/TopicDeletionManager.scala @@ -299,7 +299,7 @@ class TopicDeletionManager(controller: KafkaController, eventManager: Controller debug("Deletion started for replicas %s".format(replicasForDeletionRetry.mkString(","))) controller.replicaStateMachine.handleStateChanges(replicasForDeletionRetry, ReplicaDeletionStarted, new Callbacks.CallbackBuilder().stopReplicaCallback((stopReplicaResponseObj, replicaId) => - eventManager.put(controller.TopicDeletionStopReplicaResult(stopReplicaResponseObj, replicaId))).build) + eventManager.put(controller.TopicDeletionStopReplicaResponseReceived(stopReplicaResponseObj, replicaId))).build) if (deadReplicasForTopic.nonEmpty) { debug("Dead Replicas (%s) found for topic %s".format(deadReplicasForTopic.mkString(","), topic)) markTopicIneligibleForDeletion(Set(topic)) diff --git a/core/src/main/scala/kafka/log/Log.scala b/core/src/main/scala/kafka/log/Log.scala index 176a26855a3a..f162bc4e935f 100644 --- a/core/src/main/scala/kafka/log/Log.scala +++ b/core/src/main/scala/kafka/log/Log.scala @@ -531,6 +531,16 @@ class Log(@volatile var dir: File, } } + /** + * Close file handlers used by log but don't write to disk. This is used when the disk may have failed + */ + def closeHandlers() { + debug(s"Closing handlers of log $name") + lock synchronized { + logSegments.foreach(_.closeHandlers()) + } + } + /** * Append this message set to the active segment of the log, assigning offsets and Partition Leader Epochs * @param records The records to append diff --git a/core/src/main/scala/kafka/log/LogCleaner.scala b/core/src/main/scala/kafka/log/LogCleaner.scala index d8a86dbc8f8c..00437bc806ac 100644 --- a/core/src/main/scala/kafka/log/LogCleaner.scala +++ b/core/src/main/scala/kafka/log/LogCleaner.scala @@ -5,7 +5,7 @@ * 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 @@ -34,33 +34,34 @@ import org.apache.kafka.common.record.MemoryRecords.RecordFilter.BatchRetention import scala.collection.mutable import scala.collection.JavaConverters._ +import scala.collection.mutable.ArrayBuffer /** * The cleaner is responsible for removing obsolete records from logs which have the "compact" retention strategy. * A message with key K and offset O is obsolete if there exists a message with key K and offset O' such that O < O'. - * + * * Each log can be thought of being split into two sections of segments: a "clean" section which has previously been cleaned followed by a * "dirty" section that has not yet been cleaned. The dirty section is further divided into the "cleanable" section followed by an "uncleanable" section. * The uncleanable section is excluded from cleaning. The active log segment is always uncleanable. If there is a * compaction lag time set, segments whose largest message timestamp is within the compaction lag time of the cleaning operation are also uncleanable. * * The cleaning is carried out by a pool of background threads. Each thread chooses the dirtiest log that has the "compact" retention policy - * and cleans that. The dirtiness of the log is guessed by taking the ratio of bytes in the dirty section of the log to the total bytes in the log. - * + * and cleans that. The dirtiness of the log is guessed by taking the ratio of bytes in the dirty section of the log to the total bytes in the log. + * * To clean a log the cleaner first builds a mapping of key=>last_offset for the dirty section of the log. See kafka.log.OffsetMap for details of - * the implementation of the mapping. - * - * Once the key=>offset map is built, the log is cleaned by recopying each log segment but omitting any key that appears in the offset map with a + * the implementation of the mapping. + * + * Once the key=>offset map is built, the log is cleaned by recopying each log segment but omitting any key that appears in the offset map with a * higher offset than what is found in the segment (i.e. messages with a key that appears in the dirty section of the log). - * + * * To avoid segments shrinking to very small sizes with repeated cleanings we implement a rule by which if we will merge successive segments when * doing a cleaning if their log and index size are less than the maximum log and index size prior to the clean beginning. - * + * * Cleaned segments are swapped into the log as they become available. - * + * * One nuance that the cleaner must handle is log truncation. If a log is truncated while it is being cleaned the cleaning of that log is aborted. - * - * Messages with null payload are treated as deletes for the purpose of log compaction. This means that they receive special treatment by the cleaner. + * + * Messages with null payload are treated as deletes for the purpose of log compaction. This means that they receive special treatment by the cleaner. * The cleaner will only retain delete records for a period of time to avoid accumulating space indefinitely. This period of time is configurable on a per-topic * basis and is measured from the time the segment enters the clean portion of the log (at which point any prior message with that key has been removed). * Delete markers in the clean section of the log that are older than this time will not be retained when log segments are being recopied as part of cleaning. @@ -86,31 +87,31 @@ import scala.collection.JavaConverters._ * @param time A way to control the passage of time */ class LogCleaner(val config: CleanerConfig, - val logDirs: Array[File], + val logDirs: ArrayBuffer[File], val logs: Pool[TopicPartition, Log], time: Time = Time.SYSTEM) extends Logging with KafkaMetricsGroup { - + /* for managing the state of partitions being cleaned. package-private to allow access in tests */ private[log] val cleanerManager = new LogCleanerManager(logDirs, logs) /* a throttle used to limit the I/O of all the cleaner threads to a user-specified maximum rate */ - private val throttler = new Throttler(desiredRatePerSec = config.maxIoBytesPerSecond, - checkIntervalMs = 300, - throttleDown = true, + private val throttler = new Throttler(desiredRatePerSec = config.maxIoBytesPerSecond, + checkIntervalMs = 300, + throttleDown = true, "cleaner-io", "bytes", time = time) - + /* the threads */ private val cleaners = (0 until config.numThreads).map(new CleanerThread(_)) - + /* a metric to track the maximum utilization of any thread's buffer in the last cleaning */ - newGauge("max-buffer-utilization-percent", + newGauge("max-buffer-utilization-percent", new Gauge[Int] { def value: Int = cleaners.map(_.lastStats).map(100 * _.bufferUtilization).max.toInt }) /* a metric to track the recopy rate of each thread's last cleaning */ - newGauge("cleaner-recopy-percent", + newGauge("cleaner-recopy-percent", new Gauge[Int] { def value: Int = { val stats = cleaners.map(_.lastStats) @@ -123,7 +124,7 @@ class LogCleaner(val config: CleanerConfig, new Gauge[Int] { def value: Int = cleaners.map(_.lastStats).map(_.elapsedSecs).max.toInt }) - + /** * Start the background cleaning */ @@ -131,7 +132,7 @@ class LogCleaner(val config: CleanerConfig, info("Starting the log cleaner") cleaners.foreach(_.start()) } - + /** * Stop the background cleaning */ @@ -139,7 +140,7 @@ class LogCleaner(val config: CleanerConfig, info("Shutting down the log cleaner.") cleaners.foreach(_.shutdown()) } - + /** * Abort the cleaning of a particular partition, if it's in progress. This call blocks until the cleaning of * the partition is aborted. @@ -155,6 +156,10 @@ class LogCleaner(val config: CleanerConfig, cleanerManager.updateCheckpoints(dataDir, update=None) } + def handleLogDirFailure(dir: String) { + cleanerManager.handleLogDirFailure(dir) + } + /** * Truncate cleaner offset checkpoint for the given partition if its checkpointed offset is larger than the given offset */ @@ -197,21 +202,21 @@ class LogCleaner(val config: CleanerConfig, } isCleaned } - + /** * The cleaner threads do the actual log cleaning. Each thread processes does its cleaning repeatedly by * choosing the dirtiest log, cleaning it, and then swapping in the cleaned segments. */ private class CleanerThread(threadId: Int) extends ShutdownableThread(name = "kafka-log-cleaner-thread-" + threadId, isInterruptible = false) { - + override val loggerName = classOf[LogCleaner].getName - + if(config.dedupeBufferSize / config.numThreads > Int.MaxValue) warn("Cannot use more than 2G of cleaner buffer space per cleaner thread, ignoring excess buffer space...") val cleaner = new Cleaner(id = threadId, - offsetMap = new SkimpyOffsetMap(memory = math.min(config.dedupeBufferSize / config.numThreads, Int.MaxValue).toInt, + offsetMap = new SkimpyOffsetMap(memory = math.min(config.dedupeBufferSize / config.numThreads, Int.MaxValue).toInt, hashAlgorithm = config.hashAlgorithm), ioBufferSize = config.ioBufferSize / config.numThreads / 2, maxIoBufferSize = config.maxMessageSize, @@ -219,7 +224,7 @@ class LogCleaner(val config: CleanerConfig, throttler = throttler, time = time, checkDone = checkDone) - + @volatile var lastStats: CleanerStats = new CleanerStats() private val backOffWaitLatch = new CountDownLatch(1) @@ -241,7 +246,7 @@ class LogCleaner(val config: CleanerConfig, backOffWaitLatch.countDown() awaitShutdown() } - + /** * Clean a log if there is a dirty log available, otherwise sleep for a bit */ @@ -275,36 +280,36 @@ class LogCleaner(val config: CleanerConfig, if (!cleaned) backOffWaitLatch.await(config.backOffMs, TimeUnit.MILLISECONDS) } - + /** * Log out statistics on a single run of the cleaner. */ def recordStats(id: Int, name: String, from: Long, to: Long, stats: CleanerStats) { this.lastStats = stats def mb(bytes: Double) = bytes / (1024*1024) - val message = - "%n\tLog cleaner thread %d cleaned log %s (dirty section = [%d, %d])%n".format(id, name, from, to) + - "\t%,.1f MB of log processed in %,.1f seconds (%,.1f MB/sec).%n".format(mb(stats.bytesRead), - stats.elapsedSecs, - mb(stats.bytesRead/stats.elapsedSecs)) + - "\tIndexed %,.1f MB in %.1f seconds (%,.1f Mb/sec, %.1f%% of total time)%n".format(mb(stats.mapBytesRead), - stats.elapsedIndexSecs, - mb(stats.mapBytesRead)/stats.elapsedIndexSecs, + val message = + "%n\tLog cleaner thread %d cleaned log %s (dirty section = [%d, %d])%n".format(id, name, from, to) + + "\t%,.1f MB of log processed in %,.1f seconds (%,.1f MB/sec).%n".format(mb(stats.bytesRead), + stats.elapsedSecs, + mb(stats.bytesRead/stats.elapsedSecs)) + + "\tIndexed %,.1f MB in %.1f seconds (%,.1f Mb/sec, %.1f%% of total time)%n".format(mb(stats.mapBytesRead), + stats.elapsedIndexSecs, + mb(stats.mapBytesRead)/stats.elapsedIndexSecs, 100 * stats.elapsedIndexSecs/stats.elapsedSecs) + "\tBuffer utilization: %.1f%%%n".format(100 * stats.bufferUtilization) + - "\tCleaned %,.1f MB in %.1f seconds (%,.1f Mb/sec, %.1f%% of total time)%n".format(mb(stats.bytesRead), - stats.elapsedSecs - stats.elapsedIndexSecs, - mb(stats.bytesRead)/(stats.elapsedSecs - stats.elapsedIndexSecs), 100 * (stats.elapsedSecs - stats.elapsedIndexSecs).toDouble/stats.elapsedSecs) + + "\tCleaned %,.1f MB in %.1f seconds (%,.1f Mb/sec, %.1f%% of total time)%n".format(mb(stats.bytesRead), + stats.elapsedSecs - stats.elapsedIndexSecs, + mb(stats.bytesRead)/(stats.elapsedSecs - stats.elapsedIndexSecs), 100 * (stats.elapsedSecs - stats.elapsedIndexSecs).toDouble/stats.elapsedSecs) + "\tStart size: %,.1f MB (%,d messages)%n".format(mb(stats.bytesRead), stats.messagesRead) + - "\tEnd size: %,.1f MB (%,d messages)%n".format(mb(stats.bytesWritten), stats.messagesWritten) + - "\t%.1f%% size reduction (%.1f%% fewer messages)%n".format(100.0 * (1.0 - stats.bytesWritten.toDouble/stats.bytesRead), + "\tEnd size: %,.1f MB (%,d messages)%n".format(mb(stats.bytesWritten), stats.messagesWritten) + + "\t%.1f%% size reduction (%.1f%% fewer messages)%n".format(100.0 * (1.0 - stats.bytesWritten.toDouble/stats.bytesRead), 100.0 * (1.0 - stats.messagesWritten.toDouble/stats.messagesRead)) info(message) if (stats.invalidMessagesRead > 0) { warn("\tFound %d invalid messages during compaction.".format(stats.invalidMessagesRead)) } } - + } } @@ -327,14 +332,14 @@ private[log] class Cleaner(val id: Int, throttler: Throttler, time: Time, checkDone: (TopicPartition) => Unit) extends Logging { - + override val loggerName = classOf[LogCleaner].getName this.logIdent = "Cleaner " + id + ": " /* buffer used for read i/o */ private var readBuffer = ByteBuffer.allocate(ioBufferSize) - + /* buffer used for write i/o */ private var writeBuffer = ByteBuffer.allocate(ioBufferSize) @@ -352,7 +357,7 @@ private[log] class Cleaner(val id: Int, private[log] def clean(cleanable: LogToClean): (Long, CleanerStats) = { // figure out the timestamp below which it is safe to remove delete tombstones // this position is defined to be a configurable time beneath the last modified time of the last clean segment - val deleteHorizonMs = + val deleteHorizonMs = cleanable.log.logSegments(0, cleanable.firstDirtyOffset).lastOption match { case None => 0L case Some(seg) => seg.lastModified - cleanable.log.config.deleteRetentionMs diff --git a/core/src/main/scala/kafka/log/LogCleanerManager.scala b/core/src/main/scala/kafka/log/LogCleanerManager.scala index 6e0ebfb41363..3dca36ffda30 100755 --- a/core/src/main/scala/kafka/log/LogCleanerManager.scala +++ b/core/src/main/scala/kafka/log/LogCleanerManager.scala @@ -17,19 +17,20 @@ package kafka.log -import java.io.File +import java.io.{IOException, File} import java.util.concurrent.TimeUnit import java.util.concurrent.locks.ReentrantLock import com.yammer.metrics.core.Gauge import kafka.common.LogCleaningAbortedException import kafka.metrics.KafkaMetricsGroup -import kafka.server.checkpoints.{OffsetCheckpoint, OffsetCheckpointFile} +import kafka.server.checkpoints.OffsetCheckpointFile import kafka.utils.CoreUtils._ import kafka.utils.{Logging, Pool} import org.apache.kafka.common.TopicPartition import org.apache.kafka.common.utils.Time +import scala.collection.mutable.ArrayBuffer import scala.collection.{immutable, mutable} private[log] sealed trait LogCleaningState @@ -45,7 +46,7 @@ private[log] case object LogCleaningPaused extends LogCleaningState * While a partition is in the LogCleaningPaused state, it won't be scheduled for cleaning again, until cleaning is * requested to be resumed. */ -private[log] class LogCleanerManager(val logDirs: Array[File], val logs: Pool[TopicPartition, Log]) extends Logging with KafkaMetricsGroup { +private[log] class LogCleanerManager(val logDirs: ArrayBuffer[File], val logs: Pool[TopicPartition, Log]) extends Logging with KafkaMetricsGroup { import LogCleanerManager._ @@ -53,19 +54,19 @@ private[log] class LogCleanerManager(val logDirs: Array[File], val logs: Pool[To // package-private for testing private[log] val offsetCheckpointFile = "cleaner-offset-checkpoint" - + /* the offset checkpoints holding the last cleaned point for each log */ - private val checkpoints = logDirs.map(dir => (dir, new OffsetCheckpointFile(new File(dir, offsetCheckpointFile)))).toMap + @volatile private var checkpoints = logDirs.map(dir => (dir, new OffsetCheckpointFile(new File(dir, offsetCheckpointFile)))).toMap /* the set of logs currently being cleaned */ private val inProgress = mutable.HashMap[TopicPartition, LogCleaningState]() /* a global lock used to control all access to the in-progress set and the offset checkpoints */ private val lock = new ReentrantLock - + /* for coordinating the pausing and the cleaning of a partition */ private val pausedCleaningCond = lock.newCondition() - + /* a gauge for tracking the cleanable ratio of the dirtiest log */ @volatile private var dirtiestLogCleanableRatio = 0.0 newGauge("max-dirty-percent", new Gauge[Int] { def value = (100 * dirtiestLogCleanableRatio).toInt }) @@ -77,8 +78,21 @@ private[log] class LogCleanerManager(val logDirs: Array[File], val logs: Pool[To /** * @return the position processed for all logs. */ - def allCleanerCheckpoints: Map[TopicPartition, Long] = - checkpoints.values.flatMap(_.read()).toMap + def allCleanerCheckpoints: Map[TopicPartition, Long] = { + inLock(lock) { + checkpoints.values.flatMap(checkpoint => { + try { + checkpoint.read() + } catch { + case e: IOException => + error(s"Failed to access checkpoint file ${checkpoint.f}", e) + handleLogDirFailure(checkpoint.f.getParentFile.getAbsolutePath) + Map.empty[TopicPartition, Long] + } + }).toMap + } + } + /** * Choose the log to clean next and add it to the in-progress set. We recompute this @@ -217,19 +231,43 @@ private[log] class LogCleanerManager(val logDirs: Array[File], val logs: Pool[To def updateCheckpoints(dataDir: File, update: Option[(TopicPartition,Long)]) { inLock(lock) { val checkpoint = checkpoints(dataDir) - val existing = checkpoint.read().filterKeys(logs.keys) ++ update - checkpoint.write(existing) + if (checkpoint != null) { + try { + val existing = checkpoint.read().filterKeys(logs.keys) ++ update + checkpoint.write(existing) + } catch { + case e: IOException => + error(s"Failed to access checkpoint file ${checkpoint.f}", e) + handleLogDirFailure(checkpoint.f.getParentFile.getAbsolutePath) + } + } + } + } + + def handleLogDirFailure(dir: String) { + info(s"Stopping cleaning logs in dir $dir") + inLock(lock) { + checkpoints = checkpoints.filterKeys(_.getAbsolutePath != dir) } + info(s"Stopped cleaning logs in dir $dir") } def maybeTruncateCheckpoint(dataDir: File, topicPartition: TopicPartition, offset: Long) { inLock(lock) { if (logs.get(topicPartition).config.compact) { val checkpoint = checkpoints(dataDir) - val existing = checkpoint.read() - if (existing.getOrElse(topicPartition, 0L) > offset) - checkpoint.write(existing + (topicPartition -> offset)) + if (checkpoint != null) { + try { + val existing = checkpoint.read() + if (existing.getOrElse(topicPartition, 0L) > offset) + checkpoint.write(existing + (topicPartition -> offset)) + } catch { + case e: IOException => + error(s"Failed to access checkpoint file ${checkpoint.f}", e) + handleLogDirFailure(checkpoint.f.getParentFile.getAbsolutePath) + } + } } } } diff --git a/core/src/main/scala/kafka/log/LogManager.scala b/core/src/main/scala/kafka/log/LogManager.scala index 2df52414e3f4..f55db5ec6c7a 100755 --- a/core/src/main/scala/kafka/log/LogManager.scala +++ b/core/src/main/scala/kafka/log/LogManager.scala @@ -31,19 +31,20 @@ import org.apache.kafka.common.utils.Time import scala.collection.JavaConverters._ import scala.collection._ +import scala.collection.mutable.ArrayBuffer /** * The entry point to the kafka log management subsystem. The log manager is responsible for log creation, retrieval, and cleaning. * All read and write operations are delegated to the individual log instances. - * + * * The log manager maintains logs in one or more directories. New logs are created in the data directory * with the fewest logs. No attempt is made to move partitions after the fact or balance based on * size or I/O rate. - * + * * A background thread handles log retention by periodically truncating excess log segments. */ @threadsafe -class LogManager(val logDirs: Array[File], +class LogManager(private val logDirs: Array[File], val topicConfigs: Map[String, LogConfig], // note that this doesn't get updated after creation val defaultConfig: LogConfig, val cleanerConfig: CleanerConfig, @@ -56,6 +57,8 @@ class LogManager(val logDirs: Array[File], scheduler: Scheduler, val brokerState: BrokerState, brokerTopicStats: BrokerTopicStats, + val zkUtils: ZkUtils, + val brokerId: Int, time: Time) extends Logging { val RecoveryPointCheckpointFile = "recovery-point-offset-checkpoint" val LogStartOffsetCheckpointFile = "log-start-offset-checkpoint" @@ -66,42 +69,103 @@ class LogManager(val logDirs: Array[File], private val logs = new Pool[TopicPartition, Log]() private val logsToBeDeleted = new LinkedBlockingQueue[Log]() - createAndValidateLogDirs(logDirs) - private val dirLocks = lockLogDirs(logDirs) - private val recoveryPointCheckpoints = logDirs.map(dir => (dir, new OffsetCheckpointFile(new File(dir, RecoveryPointCheckpointFile)))).toMap - private val logStartOffsetCheckpoints = logDirs.map(dir => (dir, new OffsetCheckpointFile(new File(dir, LogStartOffsetCheckpointFile)))).toMap + val liveLogDirs: ArrayBuffer[File] = createAndValidateLogDirs(logDirs) + @volatile private var recoveryPointCheckpoints = liveLogDirs.map(dir => (dir, new OffsetCheckpointFile(new File(dir, RecoveryPointCheckpointFile)))).toMap + @volatile private var logStartOffsetCheckpoints = liveLogDirs.map(dir => (dir, new OffsetCheckpointFile(new File(dir, LogStartOffsetCheckpointFile)))).toMap + + private def offlineLogDirs = logDirs.filterNot(liveLogDirs.toSet) + loadLogs() + private val dirLocks = lockLogDirs(liveLogDirs) + // public, so we can access this from kafka.admin.DeleteTopicTest val cleaner: LogCleaner = if(cleanerConfig.enableCleaner) - new LogCleaner(cleanerConfig, logDirs, logs, time = time) + new LogCleaner(cleanerConfig, liveLogDirs, logs, time = time) else null - + + /** * Create and check validity of the given directories, specifically: *
    *
  1. Ensure that there are no duplicates in the directory list *
  2. Create each directory if it doesn't exist - *
  3. Check that each path is a readable directory + *
  4. Check that each path is a readable directory *
*/ - private def createAndValidateLogDirs(dirs: Seq[File]) { + private def createAndValidateLogDirs(dirs: Seq[File]): ArrayBuffer[File] = { if(dirs.map(_.getCanonicalPath).toSet.size < dirs.size) - throw new KafkaException("Duplicate log directory found: " + logDirs.mkString(", ")) - for(dir <- dirs) { - if(!dir.exists) { - info("Log directory '" + dir.getAbsolutePath + "' not found, creating it.") - val created = dir.mkdirs() - if(!created) - throw new KafkaException("Failed to create data directory " + dir.getAbsolutePath) + throw new KafkaException("Duplicate log directory found: " + dirs.mkString(", ")) + + val liveLogDirs = ArrayBuffer.empty[File] + + for (dir <- dirs) { + try { + if (!dir.exists) { + info("Log directory '" + dir.getAbsolutePath + "' not found, creating it.") + val created = dir.mkdirs() + if (!created) + throw new KafkaException("Failed to create data directory " + dir.getAbsolutePath) + } + if (!dir.isDirectory || !dir.canRead) + throw new KafkaException(dir.getAbsolutePath + " is not a readable log directory.") + liveLogDirs += dir + } catch { + case t: Throwable => error("Failed to create or validate data directory " + dir.getAbsolutePath, t); + } + } + if (liveLogDirs.isEmpty) + throw new KafkaException("None of the specified log dirs from " + dirs.mkString(", ") + " can be created or validated") + + liveLogDirs + } + + def handleLogDirFailure(dir: String) { + if (!logDirs.exists(_.getAbsolutePath == dir)) + throw new RuntimeException(s"Log dir $dir is not found in the config.") + + if (!liveLogDirs.exists(_.getAbsolutePath == dir)) { + info(s"Log dir $dir is already offline.") + return + } + + logCreationOrDeletionLock synchronized { + if (!liveLogDirs.exists(_.getAbsolutePath == dir)) { + info(s"Log dir $dir is already offline.") + return } - if(!dir.isDirectory || !dir.canRead) - throw new KafkaException(dir.getAbsolutePath + " is not a readable log directory.") + info(s"Stopping serving logs in dir $dir") + + val newOfflineDirs = liveLogDirs.filter(_.getAbsolutePath == dir) + liveLogDirs --= newOfflineDirs + + if (liveLogDirs.isEmpty) { + fatal(s"Shutdown broker because all log dirs in ${logDirs.mkString(", ")} have failed"); + Runtime.getRuntime().halt(1) + } + + val offlineTopicPartitions = logs.filter(_._2.dir.getParent == dir).map(_._1) + offlineTopicPartitions.foreach(topicPartition => { + val removedLog = logs.remove(topicPartition) + if (removedLog != null) { + removedLog.closeHandlers() + removedLog.removeLogMetrics() + } + info(s"Removed $topicPartition from log cache because it is in the failed log dir $dir") + }) + recoveryPointCheckpoints = recoveryPointCheckpoints.filterKeys(file => file.getAbsolutePath != dir) + logStartOffsetCheckpoints = logStartOffsetCheckpoints.filterKeys(file => file.getAbsolutePath != dir) + + if (offlineTopicPartitions.nonEmpty) + DiskUtils.propagateLogDirEvent(zkUtils, brokerId) } + info(s"Stopped serving logs in dir $dir") + if (cleaner != null) + cleaner.handleLogDirFailure(dir) } - + /** * Lock all the given directories */ @@ -109,65 +173,65 @@ class LogManager(val logDirs: Array[File], dirs.map { dir => val lock = new FileLock(new File(dir, LockFile)) if(!lock.tryLock()) - throw new KafkaException("Failed to acquire lock on file .lock in " + lock.file.getParentFile.getAbsolutePath + + throw new KafkaException("Failed to acquire lock on file .lock in " + lock.file.getParentFile.getAbsolutePath + ". A Kafka instance in another process or thread is using this directory.") lock } } - + /** * Recover and load all logs in the given data directories */ private def loadLogs(): Unit = { info("Loading logs.") val startMs = time.milliseconds - val threadPools = mutable.ArrayBuffer.empty[ExecutorService] + val threadPools = ArrayBuffer.empty[ExecutorService] val jobs = mutable.Map.empty[File, Seq[Future[_]]] - for (dir <- this.logDirs) { - val pool = Executors.newFixedThreadPool(ioThreads) - threadPools.append(pool) - - val cleanShutdownFile = new File(dir, Log.CleanShutdownFile) - - if (cleanShutdownFile.exists) { - debug( - "Found clean shutdown file. " + - "Skipping recovery for all logs in data directory: " + - dir.getAbsolutePath) - } else { - // log recovery itself is being performed by `Log` class during initialization - brokerState.newState(RecoveringFromUncleanShutdown) - } - - var recoveryPoints = Map[TopicPartition, Long]() + for (dir <- liveLogDirs) { try { - recoveryPoints = this.recoveryPointCheckpoints(dir).read - } catch { - case e: Exception => - warn("Error occurred while reading recovery-point-offset-checkpoint file of directory " + dir, e) - warn("Resetting the recovery checkpoint to 0") - } + val pool = Executors.newFixedThreadPool(ioThreads) + + val cleanShutdownFile = new File(dir, Log.CleanShutdownFile) + + if (cleanShutdownFile.exists) { + debug( + "Found clean shutdown file. " + + "Skipping recovery for all logs in data directory: " + + dir.getAbsolutePath) + } else { + // log recovery itself is being performed by `Log` class during initialization + brokerState.newState(RecoveringFromUncleanShutdown) + } - var logStartOffsets = Map[TopicPartition, Long]() - try { - logStartOffsets = this.logStartOffsetCheckpoints(dir).read - } catch { - case e: Exception => - warn("Error occurred while reading log-start-offset-checkpoint file of directory " + dir, e) - } + var recoveryPoints = Map[TopicPartition, Long]() + try { + recoveryPoints = this.recoveryPointCheckpoints(dir).read + } catch { + case e: Exception => + warn("Error occurred while reading recovery-point-offset-checkpoint file of directory " + dir, e) + warn("Resetting the recovery checkpoint to 0") + } - val jobsForDir = for { - dirContent <- Option(dir.listFiles).toList - logDir <- dirContent if logDir.isDirectory - } yield { - CoreUtils.runnable { - debug("Loading log '" + logDir.getName + "'") + var logStartOffsets = Map[TopicPartition, Long]() + try { + logStartOffsets = this.logStartOffsetCheckpoints(dir).read + } catch { + case e: Exception => + warn("Error occurred while reading log-start-offset-checkpoint file of directory " + dir, e) + } + + val jobsForDir = for { + dirContent <- Option(dir.listFiles).toList + logDir <- dirContent if logDir.isDirectory + } yield { + CoreUtils.runnable { + debug("Loading log '" + logDir.getName + "'") - val topicPartition = Log.parseTopicPartitionName(logDir) - val config = topicConfigs.getOrElse(topicPartition.topic, defaultConfig) - val logRecoveryPoint = recoveryPoints.getOrElse(topicPartition, 0L) - val logStartOffset = logStartOffsets.getOrElse(topicPartition, 0L) + val topicPartition = Log.parseTopicPartitionName(logDir) + val config = topicConfigs.getOrElse(topicPartition.topic, defaultConfig) + val logRecoveryPoint = recoveryPoints.getOrElse(topicPartition, 0L) + val logStartOffset = logStartOffsets.getOrElse(topicPartition, 0L) val current = Log( dir = logDir, @@ -191,7 +255,12 @@ class LogManager(val logDirs: Array[File], } } - jobs(cleanShutdownFile) = jobsForDir.map(pool.submit) + threadPools.append(pool) + jobs(cleanShutdownFile) = jobsForDir.map(pool.submit).toSeq + } catch { + case e: IOException => + error("Error while load log dir " + dir.getAbsolutePath, e) + } } @@ -231,7 +300,7 @@ class LogManager(val logDirs: Array[File], period = flushCheckMs, TimeUnit.MILLISECONDS) scheduler.schedule("kafka-recovery-point-checkpoint", - checkpointRecoveryPointOffsets _, + checkpointLogRecoveryOffsets _, delay = InitialTaskDelayMs, period = flushRecoveryOffsetCheckpointMs, TimeUnit.MILLISECONDS) @@ -256,7 +325,7 @@ class LogManager(val logDirs: Array[File], def shutdown() { info("Shutting down.") - val threadPools = mutable.ArrayBuffer.empty[ExecutorService] + val threadPools = ArrayBuffer.empty[ExecutorService] val jobs = mutable.Map.empty[File, Seq[Future[_]]] // stop the cleaner first @@ -265,7 +334,7 @@ class LogManager(val logDirs: Array[File], } // close logs in each dir - for (dir <- this.logDirs) { + for (dir <- liveLogDirs) { debug("Flushing and closing logs at " + dir) val pool = Executors.newFixedThreadPool(ioThreads) @@ -337,7 +406,7 @@ class LogManager(val logDirs: Array[File], } } } - checkpointRecoveryPointOffsets() + checkpointLogRecoveryOffsets() } /** @@ -357,15 +426,15 @@ class LogManager(val logDirs: Array[File], cleaner.resumeCleaning(topicPartition) } } - checkpointRecoveryPointOffsets() + checkpointLogRecoveryOffsets() } /** - * Write out the current recovery point for all logs to a text file in the log directory + * Write out the current recovery point for all logs to a text file in the log directory * to avoid recovering the whole log on startup. */ - def checkpointRecoveryPointOffsets() { - this.logDirs.foreach(checkpointLogRecoveryOffsetsInDir) + def checkpointLogRecoveryOffsets() { + liveLogDirs.foreach(checkpointLogRecoveryOffsetsInDir) } /** @@ -373,7 +442,7 @@ class LogManager(val logDirs: Array[File], * to avoid exposing data that have been deleted by DeleteRecordsRequest */ def checkpointLogStartOffsets() { - this.logDirs.foreach(checkpointLogStartOffsetsInDir) + liveLogDirs.foreach(checkpointLogStartOffsetsInDir) } /** @@ -382,7 +451,13 @@ class LogManager(val logDirs: Array[File], private def checkpointLogRecoveryOffsetsInDir(dir: File): Unit = { val recoveryPoints = this.logsByDir.get(dir.toString) if (recoveryPoints.isDefined) { - this.recoveryPointCheckpoints(dir).write(recoveryPoints.get.mapValues(_.recoveryPoint)) + try { + this.recoveryPointCheckpoints(dir).write(recoveryPoints.get.mapValues(_.recoveryPoint)) + } catch { + case e: IOException => + error("Disk error while writing to recovery point file", e); + handleLogDirFailure(dir.getAbsolutePath) + } } } @@ -392,8 +467,14 @@ class LogManager(val logDirs: Array[File], private def checkpointLogStartOffsetsInDir(dir: File): Unit = { val logs = this.logsByDir.get(dir.toString) if (logs.isDefined) { - this.logStartOffsetCheckpoints(dir).write( - logs.get.filter{case (tp, log) => log.logStartOffset > log.logSegments.head.baseOffset}.mapValues(_.logStartOffset)) + try { + this.logStartOffsetCheckpoints(dir).write( + logs.get.filter { case (tp, log) => log.logStartOffset > log.logSegments.head.baseOffset }.mapValues(_.logStartOffset)) + } catch { + case e: IOException => + error("Disk error while writing to logStartOffset file", e); + handleLogDirFailure(dir.getAbsolutePath) + } } } @@ -406,10 +487,13 @@ class LogManager(val logDirs: Array[File], * Create a log for the given topic and the given partition * If the log already exists, just return a copy of the existing log */ - def createLog(topicPartition: TopicPartition, config: LogConfig): Log = { + def createLog(topicPartition: TopicPartition, config: LogConfig, isNew: Boolean = false): Log = { logCreationOrDeletionLock synchronized { - // create the log if it has not already been created in another thread getLog(topicPartition).getOrElse { + // create the log if it has not already been created in another thread + if (!isNew && offlineLogDirs.nonEmpty) + throw new KafkaStorageException("Can not create log for " + topicPartition + " because log directories " + offlineLogDirs + " are offline") + val dataDir = nextLogDir() val dir = new File(dataDir, topicPartition.topic + "-" + topicPartition.partition) Files.createDirectories(dir.toPath) @@ -424,6 +508,7 @@ class LogManager(val logDirs: Array[File], time = time, brokerTopicStats = brokerTopicStats) logs.put(topicPartition, log) + info("Created log for partition [%s,%d] in %s with properties {%s}." .format(topicPartition.topic, topicPartition.partition, @@ -447,22 +532,26 @@ class LogManager(val logDirs: Array[File], removedLog.delete() info(s"Deleted log for partition ${removedLog.topicPartition} in ${removedLog.dir.getAbsolutePath}.") } catch { + case e: IOException => + error(s"Exception while deleting $removedLog.", e) + handleLogDirFailure(removedLog.dir.getParentFile.getAbsolutePath) case e: Throwable => - error(s"Exception in deleting $removedLog. Moving it to the end of the queue.", e) + // Dong: why do we put the log to the end of the queue instead of shutting down the broker? + error(s"Exception while deleting $removedLog. Moving it to the end of the queue.", e) failed = failed + 1 logsToBeDeleted.put(removedLog) } } } } catch { - case e: Throwable => + case e: Throwable => error(s"Exception in kafka-delete-logs thread.", e) } } /** - * Rename the directory of the given topic-partition "logdir" as "logdir.uuid.delete" and - * add it in the queue for deletion. + * Rename the directory of the given topic-partition "logdir" as "logdir.uuid.delete" and + * add it in the queue for deletion. * @param topicPartition TopicPartition that needs to be deleted */ def asyncDelete(topicPartition: TopicPartition) = { @@ -503,14 +592,14 @@ class LogManager(val logDirs: Array[File], * data directory with the fewest partitions. */ private def nextLogDir(): File = { - if(logDirs.size == 1) { - logDirs(0) + if(liveLogDirs.size == 1) { + liveLogDirs(0) } else { // count the number of logs in each parent directory (including 0 for empty directories val logCounts = allLogs.groupBy(_.dir.getParent).mapValues(_.size) - val zeros = logDirs.map(dir => (dir.getPath, 0)).toMap + val zeros = liveLogDirs.map(dir => (dir.getPath, 0)).toMap val dirCounts = (zeros ++ logCounts).toBuffer - + // choose the directory with the least logs in it val leastLoaded = dirCounts.sortBy(_._2).head new File(leastLoaded._1) @@ -609,7 +698,9 @@ object LogManager { maxPidExpirationMs = config.transactionIdExpirationMs, scheduler = kafkaScheduler, brokerState = brokerState, - time = time, - brokerTopicStats = brokerTopicStats) + brokerTopicStats = brokerTopicStats, + zkUtils = zkUtils, + brokerId = config.brokerId, + time = time) } } diff --git a/core/src/main/scala/kafka/log/LogSegment.scala b/core/src/main/scala/kafka/log/LogSegment.scala index 3e4c47def9cc..4af4d5be8881 100755 --- a/core/src/main/scala/kafka/log/LogSegment.scala +++ b/core/src/main/scala/kafka/log/LogSegment.scala @@ -480,6 +480,13 @@ class LogSegment(val log: FileRecords, CoreUtils.swallow(txnIndex.close()) } + /** + * Close file handlers used by the log segment but don't write to disk. This is used when the disk may have failed + */ + def closeHandlers() { + CoreUtils.swallow(log.closeHandlers()) + } + /** * Delete this log segment from the filesystem. * diff --git a/core/src/main/scala/kafka/server/KafkaApis.scala b/core/src/main/scala/kafka/server/KafkaApis.scala index 4f90421e5c31..7d7971277af3 100644 --- a/core/src/main/scala/kafka/server/KafkaApis.scala +++ b/core/src/main/scala/kafka/server/KafkaApis.scala @@ -144,40 +144,33 @@ class KafkaApis(val requestChannel: RequestChannel, val correlationId = request.header.correlationId val leaderAndIsrRequest = request.body[LeaderAndIsrRequest] - try { - def onLeadershipChange(updatedLeaders: Iterable[Partition], updatedFollowers: Iterable[Partition]) { - // for each new leader or follower, call coordinator to handle consumer group migration. - // this callback is invoked under the replica state change lock to ensure proper order of - // leadership changes - updatedLeaders.foreach { partition => - if (partition.topic == GROUP_METADATA_TOPIC_NAME) - groupCoordinator.handleGroupImmigration(partition.partitionId) - else if (partition.topic == TRANSACTION_STATE_TOPIC_NAME) - txnCoordinator.handleTxnImmigration(partition.partitionId, partition.getLeaderEpoch) - } - - updatedFollowers.foreach { partition => - if (partition.topic == GROUP_METADATA_TOPIC_NAME) - groupCoordinator.handleGroupEmigration(partition.partitionId) - else if (partition.topic == TRANSACTION_STATE_TOPIC_NAME) - txnCoordinator.handleTxnEmigration(partition.partitionId, partition.getLeaderEpoch) - } + def onLeadershipChange(updatedLeaders: Iterable[Partition], updatedFollowers: Iterable[Partition]) { + // for each new leader or follower, call coordinator to handle consumer group migration. + // this callback is invoked under the replica state change lock to ensure proper order of + // leadership changes + updatedLeaders.foreach { partition => + if (partition.topic == GROUP_METADATA_TOPIC_NAME) + groupCoordinator.handleGroupImmigration(partition.partitionId) + else if (partition.topic == TRANSACTION_STATE_TOPIC_NAME) + txnCoordinator.handleTxnImmigration(partition.partitionId, partition.getLeaderEpoch) } - if (authorize(request.session, ClusterAction, Resource.ClusterResource)) { - val result = replicaManager.becomeLeaderOrFollower(correlationId, leaderAndIsrRequest, onLeadershipChange) - val leaderAndIsrResponse = new LeaderAndIsrResponse(result.error, result.responseMap.asJava) - sendResponseExemptThrottle(RequestChannel.Response(request, leaderAndIsrResponse)) - } else { - val result = leaderAndIsrRequest.partitionStates.asScala.keys.map((_, Errors.CLUSTER_AUTHORIZATION_FAILED)).toMap - sendResponseMaybeThrottle(request, _ => - new LeaderAndIsrResponse(Errors.CLUSTER_AUTHORIZATION_FAILED, result.asJava)) + updatedFollowers.foreach { partition => + if (partition.topic == GROUP_METADATA_TOPIC_NAME) + groupCoordinator.handleGroupEmigration(partition.partitionId) + else if (partition.topic == TRANSACTION_STATE_TOPIC_NAME) + txnCoordinator.handleTxnEmigration(partition.partitionId, partition.getLeaderEpoch) } - } catch { - case e: FatalExitError => throw e - case e: KafkaStorageException => - fatal("Disk error during leadership change.", e) - Exit.halt(1) + } + + if (authorize(request.session, ClusterAction, Resource.ClusterResource)) { + val result = replicaManager.becomeLeaderOrFollower(correlationId, leaderAndIsrRequest, onLeadershipChange) + val leaderAndIsrResponse = new LeaderAndIsrResponse(result.error, result.responseMap.asJava) + sendResponseExemptThrottle(RequestChannel.Response(request, leaderAndIsrResponse)) + } else { + val result = leaderAndIsrRequest.partitionStates.asScala.keys.map((_, Errors.CLUSTER_AUTHORIZATION_FAILED)).toMap + sendResponseMaybeThrottle(request, _ => + new LeaderAndIsrResponse(Errors.CLUSTER_AUTHORIZATION_FAILED, result.asJava)) } } @@ -1860,7 +1853,7 @@ class KafkaApis(val requestChannel: RequestChannel, Some(new AclDeletionResult(ApiError.fromThrowable(throwable), aclBinding)) } }.asJava - + filterResponseMap.put(i, new AclFilterResponse(deletionResults)) } diff --git a/core/src/main/scala/kafka/server/KafkaServer.scala b/core/src/main/scala/kafka/server/KafkaServer.scala index 0a87750d9495..dffe6a08a104 100755 --- a/core/src/main/scala/kafka/server/KafkaServer.scala +++ b/core/src/main/scala/kafka/server/KafkaServer.scala @@ -660,9 +660,14 @@ class KafkaServer(val config: KafkaConfig, time: Time = Time.SYSTEM, threadNameP val brokerIdSet = mutable.HashSet[Int]() for (logDir <- config.logDirs) { - val brokerMetadataOpt = brokerMetadataCheckpoints(logDir).read() - brokerMetadataOpt.foreach { brokerMetadata => - brokerIdSet.add(brokerMetadata.brokerId) + try { + val brokerMetadataOpt = brokerMetadataCheckpoints(logDir).read() + brokerMetadataOpt.foreach { brokerMetadata => + brokerIdSet.add(brokerMetadata.brokerId) + } + } catch { + case e : IOException => + error(s"Fail to read ${brokerMetaPropsFile} under log directory ${logDir}", e) } } @@ -684,17 +689,16 @@ class KafkaServer(val config: KafkaConfig, time: Time = Time.SYSTEM, threadNameP } private def checkpointBrokerId(brokerId: Int) { - var logDirsWithoutMetaProps: List[String] = List() - for (logDir <- config.logDirs) { - val brokerMetadataOpt = brokerMetadataCheckpoints(logDir).read() - if(brokerMetadataOpt.isEmpty) - logDirsWithoutMetaProps ++= List(logDir) - } - - for(logDir <- logDirsWithoutMetaProps) { - val checkpoint = brokerMetadataCheckpoints(logDir) - checkpoint.write(BrokerMetadata(brokerId)) + try { + val checkpoint = brokerMetadataCheckpoints(logDir) + val brokerMetadataOpt = checkpoint.read() + if (brokerMetadataOpt.isEmpty) + checkpoint.write(BrokerMetadata(brokerId)) + } catch { + case e : IOException => + error(s"Fail to checkpoint brokerId in ${brokerMetaPropsFile} under log directory ${logDir}", e) + } } } diff --git a/core/src/main/scala/kafka/server/MetadataCache.scala b/core/src/main/scala/kafka/server/MetadataCache.scala index 466645bba7d4..a18350f72af3 100755 --- a/core/src/main/scala/kafka/server/MetadataCache.scala +++ b/core/src/main/scala/kafka/server/MetadataCache.scala @@ -31,6 +31,7 @@ import org.apache.kafka.common.internals.Topic import org.apache.kafka.common.{Node, TopicPartition} import org.apache.kafka.common.network.ListenerName import org.apache.kafka.common.protocol.Errors +import org.apache.kafka.common.requests.UpdateMetadataRequest.UpdateMetadataRequestPartitionState import org.apache.kafka.common.requests.{MetadataResponse, PartitionState, UpdateMetadataRequest} /** @@ -39,7 +40,7 @@ import org.apache.kafka.common.requests.{MetadataResponse, PartitionState, Updat */ class MetadataCache(brokerId: Int) extends Logging { private val stateChangeLogger = KafkaController.stateChangeLogger - private val cache = mutable.Map[String, mutable.Map[Int, PartitionStateInfo]]() + private val cache = mutable.Map[String, mutable.Map[Int, MetadataPartitionState]]() private var controllerId: Option[Int] = None private val aliveBrokers = mutable.Map[Int, Broker]() private val aliveNodes = mutable.Map[Int, collection.Map[ListenerName, Node]]() @@ -73,12 +74,13 @@ class MetadataCache(brokerId: Int) extends Logging { val replicas = partitionState.allReplicas val replicaInfo = getEndpoints(replicas, listenerName, errorUnavailableEndpoints) + val offlineReplicaInfo = getEndpoints(partitionState.offlineReplicas, listenerName, errorUnavailableEndpoints) maybeLeader match { case None => debug(s"Error while fetching metadata for $topicPartition: leader not available") new MetadataResponse.PartitionMetadata(Errors.LEADER_NOT_AVAILABLE, partitionId, Node.noNode(), - replicaInfo.asJava, java.util.Collections.emptyList()) + replicaInfo.asJava, java.util.Collections.emptyList(), offlineReplicaInfo.asJava) case Some(leader) => val isr = leaderAndIsr.isr @@ -89,15 +91,15 @@ class MetadataCache(brokerId: Int) extends Logging { s"following brokers ${replicas.filterNot(replicaInfo.map(_.id).contains).mkString(",")}") new MetadataResponse.PartitionMetadata(Errors.REPLICA_NOT_AVAILABLE, partitionId, leader, - replicaInfo.asJava, isrInfo.asJava) + replicaInfo.asJava, isrInfo.asJava, offlineReplicaInfo.asJava) } else if (isrInfo.size < isr.size) { debug(s"Error while fetching metadata for $topicPartition: in sync replica information not available for " + s"following brokers ${isr.filterNot(isrInfo.map(_.id).contains).mkString(",")}") new MetadataResponse.PartitionMetadata(Errors.REPLICA_NOT_AVAILABLE, partitionId, leader, - replicaInfo.asJava, isrInfo.asJava) + replicaInfo.asJava, isrInfo.asJava, offlineReplicaInfo.asJava) } else { new MetadataResponse.PartitionMetadata(Errors.NONE, partitionId, leader, replicaInfo.asJava, - isrInfo.asJava) + isrInfo.asJava, offlineReplicaInfo.asJava) } } } @@ -147,14 +149,14 @@ class MetadataCache(brokerId: Int) extends Logging { private def addOrUpdatePartitionInfo(topic: String, partitionId: Int, - stateInfo: PartitionStateInfo) { + stateInfo: MetadataPartitionState) { inWriteLock(partitionMetadataLock) { val infos = cache.getOrElseUpdate(topic, mutable.Map()) infos(partitionId) = stateInfo } } - def getPartitionInfo(topic: String, partitionId: Int): Option[PartitionStateInfo] = { + def getPartitionInfo(topic: String, partitionId: Int): Option[MetadataPartitionState] = { inReadLock(partitionMetadataLock) { cache.get(topic).flatMap(_.get(partitionId)) } @@ -223,10 +225,10 @@ class MetadataCache(brokerId: Int) extends Logging { } } - private def partitionStateToPartitionStateInfo(partitionState: PartitionState): PartitionStateInfo = { + private def partitionStateToPartitionStateInfo(partitionState: UpdateMetadataRequestPartitionState): MetadataPartitionState = { val leaderAndIsr = LeaderAndIsr(partitionState.leader, partitionState.leaderEpoch, partitionState.isr.asScala.map(_.toInt).toList, partitionState.zkVersion) val leaderInfo = LeaderIsrAndControllerEpoch(leaderAndIsr, partitionState.controllerEpoch) - PartitionStateInfo(leaderInfo, partitionState.replicas.asScala.map(_.toInt)) + MetadataPartitionState(leaderInfo, partitionState.replicas.asScala.map(_.toInt), partitionState.offlineReplicas.asScala.map(_.toInt)) } def contains(topic: String): Boolean = { diff --git a/core/src/main/scala/kafka/server/ReplicaFetcherThread.scala b/core/src/main/scala/kafka/server/ReplicaFetcherThread.scala index d7420dd5fdac..8f12fac2db28 100644 --- a/core/src/main/scala/kafka/server/ReplicaFetcherThread.scala +++ b/core/src/main/scala/kafka/server/ReplicaFetcherThread.scala @@ -18,7 +18,7 @@ package kafka.server import java.util - +import java.io.IOException import kafka.admin.AdminUtils import kafka.api.{FetchRequest => _, _} import kafka.cluster.{BrokerEndPoint, Replica} @@ -27,7 +27,6 @@ import kafka.log.LogConfig import kafka.server.ReplicaFetcherThread._ import kafka.server.epoch.LeaderEpochCache import org.apache.kafka.common.requests.EpochEndOffset._ -import kafka.utils.Exit import org.apache.kafka.common.TopicPartition import org.apache.kafka.common.internals.FatalExitError import org.apache.kafka.common.metrics.Metrics @@ -83,10 +82,10 @@ class ReplicaFetcherThread(name: String, // process fetched data def processPartitionData(topicPartition: TopicPartition, fetchOffset: Long, partitionData: PartitionData) { - try { - val replica = replicaMgr.getReplica(topicPartition).get - val records = partitionData.toRecords + val replica = replicaMgr.getReplica(topicPartition).get + val records = partitionData.toRecords + try { maybeWarnIfOversizedRecords(records, topicPartition) if (fetchOffset != replica.logEndOffset.messageOffset) @@ -114,9 +113,9 @@ class ReplicaFetcherThread(name: String, quota.record(records.sizeInBytes) replicaMgr.brokerTopicStats.updateReplicationBytesIn(records.sizeInBytes) } catch { - case e: KafkaStorageException => - fatal(s"Disk error while replicating data for $topicPartition", e) - Exit.halt(1) + case e@ (_: KafkaStorageException | _: IOException) => + error(s"Disk error while replicating data for $topicPartition", e) + replicaMgr.getLogDir(topicPartition).foreach(replicaMgr.handleLogDirFailure) } } diff --git a/core/src/main/scala/kafka/server/ReplicaManager.scala b/core/src/main/scala/kafka/server/ReplicaManager.scala index 853b7c48a492..87f452504496 100644 --- a/core/src/main/scala/kafka/server/ReplicaManager.scala +++ b/core/src/main/scala/kafka/server/ReplicaManager.scala @@ -173,7 +173,9 @@ class ReplicaManager(val config: KafkaConfig, private val replicaStateChangeLock = new Object val replicaFetcherManager = createReplicaFetcherManager(metrics, time, threadNamePrefix, quotaManager) private val highWatermarkCheckPointThreadStarted = new AtomicBoolean(false) - val highWatermarkCheckpoints = config.logDirs.map(dir => (new File(dir).getAbsolutePath, new OffsetCheckpointFile(new File(dir, ReplicaManager.HighWatermarkFilename)))).toMap + @volatile var highWatermarkCheckpoints = logManager.liveLogDirs.map(dir => + (dir.getAbsolutePath, new OffsetCheckpointFile(new File(dir.getAbsolutePath, ReplicaManager.HighWatermarkFilename)))).toMap + private var hwThreadInitialized = false this.logIdent = "[Replica Manager on Broker " + localBrokerId + "]: " val stateChangeLogger = KafkaController.stateChangeLogger @@ -317,8 +319,16 @@ class ReplicaManager(val config: KafkaConfig, // First stop fetchers for all partitions, then stop the corresponding replicas replicaFetcherManager.removeFetcherForPartitions(partitions) for (topicPartition <- partitions){ - val error = stopReplica(topicPartition, stopReplicaRequest.deletePartitions) - responseMap.put(topicPartition, error) + try { + val errorCode = stopReplica(topicPartition, stopReplicaRequest.deletePartitions) + responseMap.put(topicPartition, errorCode) + } catch { + case e@ (_: KafkaStorageException | _: IOException) => + stateChangeLogger.error(s"Broker $localBrokerId ignoring stop replica (delete=${stopReplicaRequest.deletePartitions}) for partition $topicPartition due to storage exception", e) + error("Error stopping replicas of partition %s".format(topicPartition), e) + getLogDir(topicPartition).foreach(handleLogDirFailure) + responseMap.put(topicPartition, Errors.KAFKA_STORAGE_ERROR) + } } (responseMap, Errors.NONE) } @@ -356,6 +366,13 @@ class ReplicaManager(val config: KafkaConfig, def getReplica(tp: TopicPartition): Option[Replica] = getReplica(tp, localBrokerId) + def getLogDir(topicPartition: TopicPartition): Option[String] = { + getReplica(topicPartition).flatMap(_.log) match { + case Some(log) => Some(log.dir.getParent) + case None => None + } + } + /** * Append messages to leader replicas of the partition, and wait for them to be replicated to other replicas; * the callback function will be triggered either when timeout or the required acks are satisfied; @@ -543,7 +560,14 @@ class ReplicaManager(val config: KafkaConfig, val partitionOpt = getPartition(topicPartition) val info = partitionOpt match { case Some(partition) => - partition.appendRecordsToLeader(records, isFromClient, requiredAcks) + try { + partition.appendRecordsToLeader(records, isFromClient, requiredAcks) + } catch { + case e: KafkaStorageException => + error("Error processing append operation on partition %s".format(topicPartition), e) + getLogDir(topicPartition).foreach(handleLogDirFailure) + throw e + } case None => throw new UnknownTopicOrPartitionException("Partition %s doesn't exist on %d" .format(topicPartition, localBrokerId)) @@ -567,15 +591,12 @@ class ReplicaManager(val config: KafkaConfig, } catch { // NOTE: Failed produce requests metric is not incremented for known exceptions // it is supposed to indicate un-expected failures of a broker in handling a produce request - case e: KafkaStorageException => - fatal("Halting due to unrecoverable I/O error while handling produce request: ", e) - Exit.halt(1) - (topicPartition, null) case e@ (_: UnknownTopicOrPartitionException | _: NotLeaderForPartitionException | _: RecordTooLargeException | _: RecordBatchTooLargeException | _: CorruptRecordException | + _: KafkaStorageException | _: InvalidTimestampException) => (topicPartition, LogAppendResult(LogAppendInfo.UnknownLogAppendInfo, Some(e))) case t: Throwable => @@ -713,19 +734,26 @@ class ReplicaManager(val config: KafkaConfig, val fetchTimeMs = time.milliseconds val logReadInfo = localReplica.log match { case Some(log) => - val adjustedFetchSize = math.min(partitionFetchSize, limitBytes) - - // Try the read first, this tells us whether we need all of adjustedFetchSize for this partition - val fetch = log.read(offset, adjustedFetchSize, maxOffsetOpt, minOneMessage, isolationLevel) - - // If the partition is being throttled, simply return an empty set. - if (shouldLeaderThrottle(quota, tp, replicaId)) - FetchDataInfo(fetch.fetchOffsetMetadata, MemoryRecords.EMPTY) - // For FetchRequest version 3, we replace incomplete message sets with an empty one as consumers can make - // progress in such cases and don't need to report a `RecordTooLargeException` - else if (!hardMaxBytesLimit && fetch.firstEntryIncomplete) - FetchDataInfo(fetch.fetchOffsetMetadata, MemoryRecords.EMPTY) - else fetch + try { + val adjustedFetchSize = math.min(partitionFetchSize, limitBytes) + + // Try the read first, this tells us whether we need all of adjustedFetchSize for this partition + val fetch = log.read(offset, adjustedFetchSize, maxOffsetOpt, minOneMessage, isolationLevel) + + // If the partition is being throttled, simply return an empty set. + if (shouldLeaderThrottle(quota, tp, replicaId)) + FetchDataInfo(fetch.fetchOffsetMetadata, MemoryRecords.EMPTY) + // For FetchRequest version 3, we replace incomplete message sets with an empty one as consumers can make + // progress in such cases and don't need to report a `RecordTooLargeException` + else if (!hardMaxBytesLimit && fetch.firstEntryIncomplete) + FetchDataInfo(fetch.fetchOffsetMetadata, MemoryRecords.EMPTY) + else fetch + } catch { + case e@ (_: IOException | _: KafkaStorageException) => + error(s"Error processing fetch operation on partition ${tp}, offset $offset", e) + getLogDir(tp).foreach(handleLogDirFailure) + throw new KafkaStorageException(s"Error processing fetch operation on partition ${tp}, offset $offset", e) + } case None => error(s"Leader for partition $tp does not have a local log") @@ -747,6 +775,7 @@ class ReplicaManager(val config: KafkaConfig, case e@ (_: UnknownTopicOrPartitionException | _: NotLeaderForPartitionException | _: ReplicaNotAvailableException | + _: KafkaStorageException | _: OffsetOutOfRangeException) => LogReadResult(info = FetchDataInfo(LogOffsetMetadata.UnknownOffsetMetadata, MemoryRecords.EMPTY), highWatermark = -1L, @@ -819,7 +848,8 @@ class ReplicaManager(val config: KafkaConfig, } } - def becomeLeaderOrFollower(correlationId: Int,leaderAndISRRequest: LeaderAndIsrRequest, + def becomeLeaderOrFollower(correlationId: Int, + leaderAndISRRequest: LeaderAndIsrRequest, onLeadershipChange: (Iterable[Partition], Iterable[Partition]) => Unit): BecomeLeaderOrFollowerResult = { leaderAndISRRequest.partitionStates.asScala.foreach { case (topicPartition, stateInfo) => stateChangeLogger.trace("Broker %d received LeaderAndIsr request %s correlation id %d from controller %d epoch %d for partition [%s,%d]" @@ -926,18 +956,25 @@ class ReplicaManager(val config: KafkaConfig, replicaFetcherManager.removeFetcherForPartitions(partitionState.keySet.map(_.topicPartition)) // Update the partition information to be the leader partitionState.foreach{ case (partition, partitionStateInfo) => - if (partition.makeLeader(controllerId, partitionStateInfo, correlationId)) - partitionsToMakeLeaders += partition - else - stateChangeLogger.info(("Broker %d skipped the become-leader state change after marking its partition as leader with correlation id %d from " + - "controller %d epoch %d for partition %s since it is already the leader for the partition.") - .format(localBrokerId, correlationId, controllerId, epoch, partition.topicPartition)) - } - partitionsToMakeLeaders.foreach { partition => - stateChangeLogger.trace(("Broker %d stopped fetchers as part of become-leader request from controller " + - "%d epoch %d with correlation id %d for partition %s") - .format(localBrokerId, controllerId, epoch, correlationId, partition.topicPartition)) + try { + if (partition.makeLeader(controllerId, partitionStateInfo, correlationId)) { + partitionsToMakeLeaders += partition + stateChangeLogger.trace(("Broker %d stopped fetchers as part of become-leader request from controller " + + "%d epoch %d with correlation id %d for partition %s") + .format(localBrokerId, controllerId, epoch, correlationId, partition.topicPartition)) + } else + stateChangeLogger.info(("Broker %d skipped the become-leader state change after marking its partition as leader with correlation id %d from " + + "controller %d epoch %d for partition %s since it is already the leader for the partition.") + .format(localBrokerId, correlationId, controllerId, epoch, partition.topicPartition)) + } catch { + case e@ (_: KafkaStorageException | _: IOException) => + stateChangeLogger.error(("Broker %d skipped the become-leader state change with correlation id %d from " + + "controller %d epoch %d for partition %s since the replica for the partition is offline due to disk error %s.") + .format(localBrokerId, correlationId, controllerId, epoch, partition.topicPartition, e)) + responseMap.put(new TopicPartition(partition.topic, partition.partitionId), Errors.KAFKA_STORAGE_ERROR) + } } + } catch { case e: Throwable => partitionState.keys.foreach { partition => @@ -996,27 +1033,36 @@ class ReplicaManager(val config: KafkaConfig, // TODO: Delete leaders from LeaderAndIsrRequest partitionState.foreach{ case (partition, partitionStateInfo) => - val newLeaderBrokerId = partitionStateInfo.leader - metadataCache.getAliveBrokers.find(_.id == newLeaderBrokerId) match { - // Only change partition state when the leader is available - case Some(_) => - if (partition.makeFollower(controllerId, partitionStateInfo, correlationId)) - partitionsToMakeFollower += partition - else - stateChangeLogger.info(("Broker %d skipped the become-follower state change after marking its partition as follower with correlation id %d from " + - "controller %d epoch %d for partition %s since the new leader %d is the same as the old leader") + try { + val newLeaderBrokerId = partitionStateInfo.leader + metadataCache.getAliveBrokers.find(_.id == newLeaderBrokerId) match { + // Only change partition state when the leader is available + case Some(_) => + if (partition.makeFollower(controllerId, partitionStateInfo, correlationId)) + partitionsToMakeFollower += partition + else + stateChangeLogger.info(("Broker %d skipped the become-follower state change after marking its partition as follower with correlation id %d from " + + "controller %d epoch %d for partition %s since the new leader %d is the same as the old leader") + .format(localBrokerId, correlationId, controllerId, partitionStateInfo.controllerEpoch, + partition.topicPartition, newLeaderBrokerId)) + case None => + // The leader broker should always be present in the metadata cache. + // If not, we should record the error message and abort the transition process for this partition + stateChangeLogger.error(("Broker %d received LeaderAndIsrRequest with correlation id %d from controller" + + " %d epoch %d for partition %s but cannot become follower since the new leader %d is unavailable.") .format(localBrokerId, correlationId, controllerId, partitionStateInfo.controllerEpoch, - partition.topicPartition, newLeaderBrokerId)) - case None => - // The leader broker should always be present in the metadata cache. - // If not, we should record the error message and abort the transition process for this partition - stateChangeLogger.error(("Broker %d received LeaderAndIsrRequest with correlation id %d from controller" + - " %d epoch %d for partition %s but cannot become follower since the new leader %d is unavailable.") + partition.topicPartition, newLeaderBrokerId)) + // Create the local replica even if the leader is unavailable. This is required to ensure that we include + // the partition's high watermark in the checkpoint file (see KAFKA-1647) + partition.getOrCreateReplica(isNew = partitionStateInfo.isNew) + } + } catch { + case e@ (_: KafkaStorageException | _: IOException) => + stateChangeLogger.error(("Broker %d skipped the become-follower state change with correlation id %d from " + + "controller %d epoch %d for partition [%s,%d] since the replica for the partition is offline due to disk error %s") .format(localBrokerId, correlationId, controllerId, partitionStateInfo.controllerEpoch, - partition.topicPartition, newLeaderBrokerId)) - // Create the local replica even if the leader is unavailable. This is required to ensure that we include - // the partition's high watermark in the checkpoint file (see KAFKA-1647) - partition.getOrCreateReplica() + partition.topic, partition.partitionId, e)) + responseMap.put(new TopicPartition(partition.topic, partition.partitionId), Errors.KAFKA_STORAGE_ERROR) } } @@ -1118,10 +1164,38 @@ class ReplicaManager(val config: KafkaConfig, highWatermarkCheckpoints(dir).write(hwms) } catch { case e: IOException => - fatal("Error writing to highwatermark file: ", e) - Exit.halt(1) + error("Error writing to highwatermark file: ", e) + handleLogDirFailure(dir) + } + } + } + + def handleLogDirFailure(dir: String) { + info(s"Stopping serving replicas in dir $dir") + + replicaStateChangeLock synchronized { + val newOfflinePartitions = allPartitions.values.filter { partition => + partition.getReplica(config.brokerId) match { + case Some(replica) => + replica.log.isDefined && replica.log.get.dir.getParentFile.getAbsolutePath == dir + case None => false + } + }.map(_.topicPartition) + + newOfflinePartitions.foreach { topicPartition => + val partition = allPartitions.remove(topicPartition) + partition.removePartitionMetrics() + val topicHasPartitions = allPartitions.keys.exists(tp => topicPartition.topic == tp.topic) + if (!topicHasPartitions) + brokerTopicStats.removeMetrics(topicPartition.topic) } + replicaFetcherManager.removeFetcherForPartitions(newOfflinePartitions.toSet) + highWatermarkCheckpoints = highWatermarkCheckpoints.filterKeys(_ != dir) + stateChangeLogger.warn("Broker %d stopped fetcher for partitions %s because they are in the failed log dir %s" + .format(localBrokerId, newOfflinePartitions.mkString(", "), dir)) } + info(s"Stopped serving replicas in dir $dir") + logManager.handleLogDirFailure(dir) } // High watermark do not need to be checkpointed only when under unit tests diff --git a/core/src/main/scala/kafka/server/checkpoints/CheckpointFile.scala b/core/src/main/scala/kafka/server/checkpoints/CheckpointFile.scala index cc5062063947..c319a2577659 100644 --- a/core/src/main/scala/kafka/server/checkpoints/CheckpointFile.scala +++ b/core/src/main/scala/kafka/server/checkpoints/CheckpointFile.scala @@ -33,7 +33,7 @@ class CheckpointFile[T](val file: File, version: Int, formatter: CheckpointFileF private val path = file.toPath.toAbsolutePath private val tempPath = Paths.get(path.toString + ".tmp") private val lock = new Object() - + try Files.createFile(file.toPath) // create the file if it doesn't exist catch { case _: FileAlreadyExistsException => } @@ -56,13 +56,6 @@ class CheckpointFile[T](val file: File, version: Int, formatter: CheckpointFileF writer.flush() fileOutputStream.getFD().sync() - } catch { - case e: FileNotFoundException => - if (FileSystems.getDefault.isReadOnly) { - fatal(s"Halting writes to checkpoint file (${file.getAbsolutePath}) because the underlying file system is inaccessible: ", e) - Exit.halt(1) - } - throw e } finally { writer.close() } diff --git a/core/src/main/scala/kafka/utils/DiskUtils.scala b/core/src/main/scala/kafka/utils/DiskUtils.scala new file mode 100644 index 000000000000..cb1dc4902fdc --- /dev/null +++ b/core/src/main/scala/kafka/utils/DiskUtils.scala @@ -0,0 +1,58 @@ +/** + * 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.utils + +import kafka.controller.{LogDirEventNotificationListener} +import scala.collection.{Map, Seq, Set} + +object DiskUtils extends Logging { + + private val LogDirEventNotificationPrefix = "log_dir_event_" + val LogDirFailureEvent = 1 + + def propagateLogDirEvent(zkUtils: ZkUtils, brokerId: Int) { + val logDirEventNotificationPath: String = zkUtils.createSequentialPersistentPath( + ZkUtils.LogDirEventNotificationPath + "/" + LogDirEventNotificationPrefix, logDirFailureEventZkData(brokerId)) + debug("Added " + logDirEventNotificationPath + " for broker " + brokerId) + } + + private def logDirFailureEventZkData(brokerId: Int): String = { + Json.encode(Map("version" -> LogDirEventNotificationListener.version, "broker" -> brokerId, "event" -> LogDirFailureEvent)) + } + + def getBrokerIdFromLogDirEvent(zkUtils: ZkUtils, child: String): Option[Int] = { + val changeZnode = ZkUtils.LogDirEventNotificationPath + "/" + child + val (jsonOpt, stat) = zkUtils.readDataMaybeNull(changeZnode) + if (jsonOpt.isDefined) { + val json = Json.parseFull(jsonOpt.get) + + json match { + case Some(m) => + val brokerAndTopics = m.asInstanceOf[Map[String, Any]] + val brokerId = brokerAndTopics.get("broker").get.asInstanceOf[Int] + Some(brokerId) + case None => + error("Invalid topic and partition JSON: " + jsonOpt.get + " in ZK: " + changeZnode) + None + } + } else { + None + } + } + +} diff --git a/core/src/main/scala/kafka/utils/ZkUtils.scala b/core/src/main/scala/kafka/utils/ZkUtils.scala index e03893c7e764..e9a8e317985c 100644 --- a/core/src/main/scala/kafka/utils/ZkUtils.scala +++ b/core/src/main/scala/kafka/utils/ZkUtils.scala @@ -51,6 +51,7 @@ object ZkUtils { val ControllerPath = "/controller" val ControllerEpochPath = "/controller_epoch" val IsrChangeNotificationPath = "/isr_change_notification" + val LogDirEventNotificationPath = "/log_dir_event_notification" val KafkaAclPath = "/kafka-acl" val KafkaAclChangesPath = "/kafka-acl-changes" @@ -75,7 +76,8 @@ object ZkUtils { IsrChangeNotificationPath, KafkaAclPath, KafkaAclChangesPath, - ProducerIdBlockPath) + ProducerIdBlockPath, + LogDirEventNotificationPath) // Important: it is necessary to add any new top level Zookeeper path that contains // sensitive information that should not be world readable to the Seq @@ -239,7 +241,8 @@ class ZkUtils(val zkClient: ZkClient, DeleteTopicsPath, BrokerSequenceIdPath, IsrChangeNotificationPath, - ProducerIdBlockPath) + ProducerIdBlockPath, + LogDirEventNotificationPath) // Visible for testing val zkPath = new ZkPath(zkClient) diff --git a/core/src/test/scala/integration/kafka/api/AuthorizerIntegrationTest.scala b/core/src/test/scala/integration/kafka/api/AuthorizerIntegrationTest.scala index 09ff9be80d81..71c0f42cd052 100644 --- a/core/src/test/scala/integration/kafka/api/AuthorizerIntegrationTest.scala +++ b/core/src/test/scala/integration/kafka/api/AuthorizerIntegrationTest.scala @@ -48,6 +48,7 @@ import org.junit.{After, Assert, Before, Test} import scala.collection.JavaConverters._ import scala.collection.mutable import scala.collection.mutable.Buffer +import org.apache.kafka.common.requests.UpdateMetadataRequest.UpdateMetadataRequestPartitionState class AuthorizerIntegrationTest extends BaseRequestTest { @@ -272,7 +273,8 @@ class AuthorizerIntegrationTest extends BaseRequestTest { } private def createUpdateMetadataRequest = { - val partitionState = Map(tp -> new PartitionState(Int.MaxValue, brokerId, Int.MaxValue, List(brokerId).asJava, 2, Seq(brokerId).asJava)).asJava + val partitionState = Map(tp -> new UpdateMetadataRequestPartitionState( + Int.MaxValue, brokerId, Int.MaxValue, List(brokerId).asJava, 2, Seq(brokerId).asJava, Seq.empty[Integer].asJava)).asJava val securityProtocol = SecurityProtocol.PLAINTEXT val brokers = Set(new requests.UpdateMetadataRequest.Broker(brokerId, Seq(new requests.UpdateMetadataRequest.EndPoint("localhost", 0, securityProtocol, @@ -304,7 +306,7 @@ class AuthorizerIntegrationTest extends BaseRequestTest { private def leaderAndIsrRequest = { new requests.LeaderAndIsrRequest.Builder(brokerId, Int.MaxValue, - Map(tp -> new PartitionState(Int.MaxValue, brokerId, Int.MaxValue, List(brokerId).asJava, 2, Seq(brokerId).asJava)).asJava, + Map(tp -> new PartitionState(Int.MaxValue, brokerId, Int.MaxValue, List(brokerId).asJava, 2, Seq(brokerId).asJava, false)).asJava, Set(new Node(brokerId, "localhost", 0)).asJava).build() } diff --git a/core/src/test/scala/integration/kafka/api/TransactionsTest.scala b/core/src/test/scala/integration/kafka/api/TransactionsTest.scala index 0e57e53bc4dd..760cc39a974f 100644 --- a/core/src/test/scala/integration/kafka/api/TransactionsTest.scala +++ b/core/src/test/scala/integration/kafka/api/TransactionsTest.scala @@ -389,7 +389,7 @@ class TransactionsTest extends KafkaServerTestHarness { val recordMetadata = result.get() error(s"Missed a producer fenced exception when writing to ${recordMetadata.topic}-${recordMetadata.partition}. Grab the logs!!") servers.foreach { server => - error(s"log dirs: ${server.logManager.logDirs.map(_.getAbsolutePath).head}") + error(s"log dirs: ${server.logManager.liveLogDirs.map(_.getAbsolutePath).head}") } fail("Should not be able to send messages from a fenced producer.") } catch { @@ -436,7 +436,7 @@ class TransactionsTest extends KafkaServerTestHarness { val recordMetadata = result.get() error(s"Missed a producer fenced exception when writing to ${recordMetadata.topic}-${recordMetadata.partition}. Grab the logs!!") servers.foreach { case (server) => - error(s"log dirs: ${server.logManager.logDirs.map(_.getAbsolutePath).head}") + error(s"log dirs: ${server.logManager.liveLogDirs.map(_.getAbsolutePath).head}") } fail("Should not be able to send messages from a fenced producer.") } catch { diff --git a/core/src/test/scala/unit/kafka/log/AbstractLogCleanerIntegrationTest.scala b/core/src/test/scala/unit/kafka/log/AbstractLogCleanerIntegrationTest.scala index bf361992c59c..2e1cf85f2a98 100644 --- a/core/src/test/scala/unit/kafka/log/AbstractLogCleanerIntegrationTest.scala +++ b/core/src/test/scala/unit/kafka/log/AbstractLogCleanerIntegrationTest.scala @@ -26,7 +26,7 @@ import org.apache.kafka.common.TopicPartition import org.apache.kafka.common.utils.Utils import org.junit.After -import scala.collection.mutable.ListBuffer +import scala.collection.mutable.{ArrayBuffer, ListBuffer} abstract class AbstractLogCleanerIntegrationTest { @@ -108,7 +108,7 @@ abstract class AbstractLogCleanerIntegrationTest { maxMessageSize = maxMessageSize, backOffMs = backOffMs) new LogCleaner(cleanerConfig, - logDirs = Array(logDir), + logDirs = ArrayBuffer(logDir), logs = logMap, time = time) } diff --git a/core/src/test/scala/unit/kafka/log/LogCleanerIntegrationTest.scala b/core/src/test/scala/unit/kafka/log/LogCleanerIntegrationTest.scala index 8a119c2f51aa..e569b293a837 100755 --- a/core/src/test/scala/unit/kafka/log/LogCleanerIntegrationTest.scala +++ b/core/src/test/scala/unit/kafka/log/LogCleanerIntegrationTest.scala @@ -5,7 +5,7 @@ * 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 diff --git a/core/src/test/scala/unit/kafka/log/LogCleanerManagerTest.scala b/core/src/test/scala/unit/kafka/log/LogCleanerManagerTest.scala index b4c1790f2137..8fed9f165ac7 100644 --- a/core/src/test/scala/unit/kafka/log/LogCleanerManagerTest.scala +++ b/core/src/test/scala/unit/kafka/log/LogCleanerManagerTest.scala @@ -29,6 +29,8 @@ import org.junit.Assert._ import org.junit.{After, Test} import org.scalatest.junit.JUnitSuite +import scala.collection.mutable.ArrayBuffer + /** * Unit tests for the log cleaning logic */ @@ -218,7 +220,7 @@ class LogCleanerManagerTest extends JUnitSuite with Logging { private def createCleanerManager(log: Log): LogCleanerManager = { val logs = new Pool[TopicPartition, Log]() logs.put(new TopicPartition("log", 0), log) - val cleanerManager = new LogCleanerManager(Array(logDir), logs) + val cleanerManager = new LogCleanerManager(ArrayBuffer(logDir), logs) cleanerManager } diff --git a/core/src/test/scala/unit/kafka/log/LogManagerTest.scala b/core/src/test/scala/unit/kafka/log/LogManagerTest.scala index 8b7819f030ba..891cd8fb570a 100755 --- a/core/src/test/scala/unit/kafka/log/LogManagerTest.scala +++ b/core/src/test/scala/unit/kafka/log/LogManagerTest.scala @@ -24,6 +24,7 @@ import kafka.common._ import kafka.server.FetchDataInfo import kafka.server.checkpoints.OffsetCheckpointFile import kafka.utils._ +import kafka.zk.ZooKeeperTestHarness import org.apache.kafka.common.TopicPartition import org.apache.kafka.common.errors.OffsetOutOfRangeException import org.apache.kafka.common.requests.IsolationLevel @@ -31,7 +32,7 @@ import org.apache.kafka.common.utils.Utils import org.junit.Assert._ import org.junit.{After, Before, Test} -class LogManagerTest { +class LogManagerTest extends ZooKeeperTestHarness { val time: MockTime = new MockTime() val maxRollInterval = 100 @@ -48,19 +49,21 @@ class LogManagerTest { val veryLargeLogFlushInterval = 10000000L @Before - def setUp() { + override def setUp() { + super.setUp() logDir = TestUtils.tempDir() logManager = createLogManager() logManager.startup() - logDir = logManager.logDirs(0) + logDir = logManager.liveLogDirs(0) } @After - def tearDown() { + override def tearDown() { + super.tearDown() if(logManager != null) logManager.shutdown() Utils.delete(logDir) - logManager.logDirs.foreach(Utils.delete) + logManager.liveLogDirs.foreach(Utils.delete) } /** @@ -264,7 +267,7 @@ class LogManagerTest { logManager.shutdown() logDir = TestUtils.tempDir() logManager = TestUtils.createLogManager( - logDirs = Array(new File(logDir.getAbsolutePath + File.separator))) + logDirs = Array(new File(logDir.getAbsolutePath + File.separator)), zkUtils = zkUtils) logManager.startup() verifyCheckpointRecovery(Seq(new TopicPartition("test-a", 1)), logManager) } @@ -294,7 +297,7 @@ class LogManagerTest { log.flush() }) - logManager.checkpointRecoveryPointOffsets() + logManager.checkpointLogRecoveryOffsets() val checkpoints = new OffsetCheckpointFile(new File(logDir, logManager.RecoveryPointCheckpointFile)).read() topicPartitions.zip(logs).foreach { @@ -306,6 +309,7 @@ class LogManagerTest { private def createLogManager(logDirs: Array[File] = Array(this.logDir)): LogManager = { TestUtils.createLogManager( + zkUtils = zkUtils, defaultConfig = logConfig, logDirs = logDirs, time = this.time) diff --git a/core/src/test/scala/unit/kafka/server/HighwatermarkPersistenceTest.scala b/core/src/test/scala/unit/kafka/server/HighwatermarkPersistenceTest.scala index b6b40c21f796..f93141df4a89 100755 --- a/core/src/test/scala/unit/kafka/server/HighwatermarkPersistenceTest.scala +++ b/core/src/test/scala/unit/kafka/server/HighwatermarkPersistenceTest.scala @@ -35,24 +35,25 @@ class HighwatermarkPersistenceTest { val configs = TestUtils.createBrokerConfigs(2, TestUtils.MockZkConnect).map(KafkaConfig.fromProps) val topic = "foo" + val zkUtils = EasyMock.createMock(classOf[ZkUtils]) val logManagers = configs map { config => TestUtils.createLogManager( logDirs = config.logDirs.map(new File(_)).toArray, + zkUtils = zkUtils, cleanerConfig = CleanerConfig()) } - + @After def teardown() { - for(manager <- logManagers; dir <- manager.logDirs) + for(manager <- logManagers; dir <- manager.liveLogDirs) Utils.delete(dir) } @Test def testHighWatermarkPersistenceSinglePartition() { // mock zkclient - val zkUtils = EasyMock.createMock(classOf[ZkUtils]) EasyMock.replay(zkUtils) - + // create kafka scheduler val scheduler = new KafkaScheduler(2) scheduler.startup @@ -96,7 +97,6 @@ class HighwatermarkPersistenceTest { val topic1 = "foo1" val topic2 = "foo2" // mock zkclient - val zkUtils = EasyMock.createMock(classOf[ZkUtils]) EasyMock.replay(zkUtils) // create kafka scheduler val scheduler = new KafkaScheduler(2) @@ -163,5 +163,5 @@ class HighwatermarkPersistenceTest { replicaManager.highWatermarkCheckpoints(new File(replicaManager.config.logDirs.head).getAbsolutePath).read.getOrElse( new TopicPartition(topic, partition), 0L) } - + } diff --git a/core/src/test/scala/unit/kafka/server/ISRExpirationTest.scala b/core/src/test/scala/unit/kafka/server/ISRExpirationTest.scala index 5d221fe599e2..79a9a4625889 100644 --- a/core/src/test/scala/unit/kafka/server/ISRExpirationTest.scala +++ b/core/src/test/scala/unit/kafka/server/ISRExpirationTest.scala @@ -16,13 +16,13 @@ */ package kafka.server +import java.io.File import java.util.Properties import java.util.concurrent.atomic.AtomicBoolean import kafka.cluster.{Partition, Replica} import kafka.log.Log -import kafka.server.checkpoints.{LeaderEpochCheckpointFile, LeaderEpochFile} -import kafka.server.epoch.{LeaderEpochCache, LeaderEpochFileCache} +import kafka.server.epoch.LeaderEpochCache import kafka.utils._ import org.apache.kafka.common.TopicPartition import org.apache.kafka.common.metrics.Metrics @@ -32,7 +32,7 @@ import org.easymock.EasyMock import org.junit.Assert._ import org.junit.{After, Before, Test} -import scala.collection.mutable.{HashMap, Map} +import scala.collection.mutable.{ArrayBuffer, HashMap, Map} class IsrExpirationTest { @@ -54,9 +54,12 @@ class IsrExpirationTest { @Before def setUp() { - replicaManager = new ReplicaManager(configs.head, metrics, time, null, null, null, new AtomicBoolean(false), - QuotaFactory.instantiate(configs.head, metrics, time).follower, new BrokerTopicStats, - new MetadataCache(configs.head.brokerId)) + val logManager = EasyMock.createMock(classOf[kafka.log.LogManager]) + EasyMock.expect(logManager.liveLogDirs).andReturn(ArrayBuffer.empty[File]).anyTimes() + EasyMock.replay(logManager) + + replicaManager = new ReplicaManager(configs.head, metrics, time, null, null, logManager, new AtomicBoolean(false), + QuotaFactory.instantiate(configs.head, metrics, time).follower, new BrokerTopicStats, new MetadataCache(configs.head.brokerId)) } @After diff --git a/core/src/test/scala/unit/kafka/server/LeaderElectionTest.scala b/core/src/test/scala/unit/kafka/server/LeaderElectionTest.scala index 4ebb17bddc38..d59f1da391c7 100755 --- a/core/src/test/scala/unit/kafka/server/LeaderElectionTest.scala +++ b/core/src/test/scala/unit/kafka/server/LeaderElectionTest.scala @@ -145,7 +145,7 @@ class LeaderElectionTest extends ZooKeeperTestHarness { val partitionStates = Map( new TopicPartition(topic, partitionId) -> new PartitionState(2, brokerId2, LeaderAndIsr.initialLeaderEpoch, Seq(brokerId1, brokerId2).map(Integer.valueOf).asJava, LeaderAndIsr.initialZKVersion, - Seq(0, 1).map(Integer.valueOf).asJava) + Seq(0, 1).map(Integer.valueOf).asJava, false) ) val requestBuilder = new LeaderAndIsrRequest.Builder( controllerId, staleControllerEpoch, partitionStates.asJava, nodes.toSet.asJava) diff --git a/core/src/test/scala/unit/kafka/server/MetadataCacheTest.scala b/core/src/test/scala/unit/kafka/server/MetadataCacheTest.scala index d9fe99572a36..e00945f37387 100644 --- a/core/src/test/scala/unit/kafka/server/MetadataCacheTest.scala +++ b/core/src/test/scala/unit/kafka/server/MetadataCacheTest.scala @@ -24,7 +24,7 @@ import org.apache.kafka.common.TopicPartition import org.apache.kafka.common.network.ListenerName import org.apache.kafka.common.protocol.{ApiKeys, Errors, SecurityProtocol} import org.apache.kafka.common.requests.{PartitionState, UpdateMetadataRequest} -import org.apache.kafka.common.requests.UpdateMetadataRequest.{Broker, EndPoint} +import org.apache.kafka.common.requests.UpdateMetadataRequest.{Broker, EndPoint, UpdateMetadataRequestPartitionState} import org.junit.Test import org.junit.Assert._ @@ -65,9 +65,9 @@ class MetadataCacheTest { }.toSet val partitionStates = Map( - new TopicPartition(topic0, 0) -> new PartitionState(controllerEpoch, 0, 0, asList(0, 1, 3), zkVersion, asList(0, 1, 3)), - new TopicPartition(topic0, 1) -> new PartitionState(controllerEpoch, 1, 1, asList(1, 0), zkVersion, asList(1, 2, 0, 4)), - new TopicPartition(topic1, 0) -> new PartitionState(controllerEpoch, 2, 2, asList(2, 1), zkVersion, asList(2, 1, 3))) + new TopicPartition(topic0, 0) -> new UpdateMetadataRequestPartitionState(controllerEpoch, 0, 0, asList(0, 1, 3), zkVersion, asList(0, 1, 3), asList()), + new TopicPartition(topic0, 1) -> new UpdateMetadataRequestPartitionState(controllerEpoch, 1, 1, asList(1, 0), zkVersion, asList(1, 2, 0, 4), asList()), + new TopicPartition(topic1, 0) -> new UpdateMetadataRequestPartitionState(controllerEpoch, 2, 2, asList(2, 1), zkVersion, asList(2, 1, 3), asList())) val version = ApiKeys.UPDATE_METADATA_KEY.latestVersion val updateMetadataRequest = new UpdateMetadataRequest.Builder(version, controllerId, controllerEpoch, @@ -125,7 +125,7 @@ class MetadataCacheTest { val leader = 1 val leaderEpoch = 1 val partitionStates = Map( - new TopicPartition(topic, 0) -> new PartitionState(controllerEpoch, leader, leaderEpoch, asList(0), zkVersion, asList(0))) + new TopicPartition(topic, 0) -> new UpdateMetadataRequestPartitionState(controllerEpoch, leader, leaderEpoch, asList(0), zkVersion, asList(0), asList())) val version = ApiKeys.UPDATE_METADATA_KEY.latestVersion val updateMetadataRequest = new UpdateMetadataRequest.Builder(version, controllerId, controllerEpoch, @@ -169,7 +169,7 @@ class MetadataCacheTest { val isr = asList[Integer](0) val partitionStates = Map( - new TopicPartition(topic, 0) -> new PartitionState(controllerEpoch, leader, leaderEpoch, isr, zkVersion, replicas)) + new TopicPartition(topic, 0) -> new UpdateMetadataRequestPartitionState(controllerEpoch, leader, leaderEpoch, isr, zkVersion, replicas, asList())) val version = ApiKeys.UPDATE_METADATA_KEY.latestVersion val updateMetadataRequest = new UpdateMetadataRequest.Builder(version, controllerId, controllerEpoch, @@ -229,7 +229,7 @@ class MetadataCacheTest { val isr = asList[Integer](0, 1) val partitionStates = Map( - new TopicPartition(topic, 0) -> new PartitionState(controllerEpoch, leader, leaderEpoch, isr, zkVersion, replicas)) + new TopicPartition(topic, 0) -> new UpdateMetadataRequestPartitionState(controllerEpoch, leader, leaderEpoch, isr, zkVersion, replicas, asList())) val version = ApiKeys.UPDATE_METADATA_KEY.latestVersion val updateMetadataRequest = new UpdateMetadataRequest.Builder(version, controllerId, controllerEpoch, @@ -282,7 +282,7 @@ class MetadataCacheTest { val replicas = asList[Integer](0) val isr = asList[Integer](0, 1) val partitionStates = Map( - new TopicPartition(topic, 0) -> new PartitionState(controllerEpoch, leader, leaderEpoch, isr, 3, replicas)) + new TopicPartition(topic, 0) -> new UpdateMetadataRequestPartitionState(controllerEpoch, leader, leaderEpoch, isr, 3, replicas, asList())) val version = ApiKeys.UPDATE_METADATA_KEY.latestVersion val updateMetadataRequest = new UpdateMetadataRequest.Builder(version, 2, controllerEpoch, partitionStates.asJava, brokers.asJava).build() @@ -315,7 +315,7 @@ class MetadataCacheTest { val replicas = asList[Integer](0) val isr = asList[Integer](0, 1) val partitionStates = Map( - new TopicPartition(topic, 0) -> new PartitionState(controllerEpoch, leader, leaderEpoch, isr, 3, replicas)) + new TopicPartition(topic, 0) -> new UpdateMetadataRequestPartitionState(controllerEpoch, leader, leaderEpoch, isr, 3, replicas, asList())) val version = ApiKeys.UPDATE_METADATA_KEY.latestVersion val updateMetadataRequest = new UpdateMetadataRequest.Builder(version, 2, controllerEpoch, partitionStates.asJava, brokers.asJava).build() diff --git a/core/src/test/scala/unit/kafka/server/ReplicaManagerQuotasTest.scala b/core/src/test/scala/unit/kafka/server/ReplicaManagerQuotasTest.scala index 2ee08a225aab..4c7a1ee5f74a 100644 --- a/core/src/test/scala/unit/kafka/server/ReplicaManagerQuotasTest.scala +++ b/core/src/test/scala/unit/kafka/server/ReplicaManagerQuotasTest.scala @@ -16,6 +16,7 @@ */ package kafka.server +import java.io.File import java.util.Properties import java.util.concurrent.atomic.AtomicBoolean @@ -33,6 +34,7 @@ import org.junit.Assert._ import org.junit.{After, Test} import scala.collection.JavaConverters._ +import scala.collection.mutable.ArrayBuffer class ReplicaManagerQuotasTest { val configs = TestUtils.createBrokerConfigs(2, TestUtils.MockZkConnect).map(KafkaConfig.fromProps(_, new Properties())) @@ -178,6 +180,7 @@ class ReplicaManagerQuotasTest { //Return the same log for each partition as it doesn't matter expect(logManager.getLog(anyObject())).andReturn(Some(log)).anyTimes() + expect(logManager.liveLogDirs).andReturn(ArrayBuffer.empty[File]).anyTimes() replay(logManager) replicaManager = new ReplicaManager(configs.head, metrics, time, zkUtils, scheduler, logManager, diff --git a/core/src/test/scala/unit/kafka/server/ReplicaManagerTest.scala b/core/src/test/scala/unit/kafka/server/ReplicaManagerTest.scala index 57948544e97e..32024d6c03f4 100644 --- a/core/src/test/scala/unit/kafka/server/ReplicaManagerTest.scala +++ b/core/src/test/scala/unit/kafka/server/ReplicaManagerTest.scala @@ -5,7 +5,7 @@ * 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 @@ -25,6 +25,7 @@ import kafka.log.LogConfig import kafka.utils.{MockScheduler, MockTime, TestUtils, ZkUtils} import TestUtils.createBroker import kafka.utils.timer.MockTimer +import kafka.zk.ZooKeeperTestHarness import org.I0Itec.zkclient.ZkClient import org.apache.kafka.common.metrics.Metrics import org.apache.kafka.common.protocol.Errors @@ -41,22 +42,20 @@ import org.junit.{After, Before, Test} import scala.collection.JavaConverters._ import scala.collection.Map -class ReplicaManagerTest { +class ReplicaManagerTest extends ZooKeeperTestHarness { val topic = "test-topic" val time = new MockTime val metrics = new Metrics - var zkClient : ZkClient = _ - var zkUtils : ZkUtils = _ - + @Before - def setUp() { - zkClient = EasyMock.createMock(classOf[ZkClient]) - zkUtils = ZkUtils(zkClient, isZkSecurityEnabled = false) + override def setUp() { + super.setUp() } - + @After - def tearDown() { + override def tearDown() { + super.tearDown() metrics.close() } @@ -64,7 +63,7 @@ class ReplicaManagerTest { def testHighWaterMarkDirectoryMapping() { val props = TestUtils.createBrokerConfig(1, TestUtils.MockZkConnect) val config = KafkaConfig.fromProps(props) - val mockLogMgr = TestUtils.createLogManager(config.logDirs.map(new File(_)).toArray) + val mockLogMgr = TestUtils.createLogManager(config.logDirs.map(new File(_)).toArray, zkUtils) val rm = new ReplicaManager(config, metrics, time, zkUtils, new MockScheduler(time), mockLogMgr, new AtomicBoolean(false), QuotaFactory.instantiate(config, metrics, time).follower, new BrokerTopicStats, new MetadataCache(config.brokerId)) @@ -83,7 +82,7 @@ class ReplicaManagerTest { val props = TestUtils.createBrokerConfig(1, TestUtils.MockZkConnect) props.put("log.dir", TestUtils.tempRelativeDir("data").getAbsolutePath) val config = KafkaConfig.fromProps(props) - val mockLogMgr = TestUtils.createLogManager(config.logDirs.map(new File(_)).toArray) + val mockLogMgr = TestUtils.createLogManager(config.logDirs.map(new File(_)).toArray, zkUtils) val rm = new ReplicaManager(config, metrics, time, zkUtils, new MockScheduler(time), mockLogMgr, new AtomicBoolean(false), QuotaFactory.instantiate(config, metrics, time).follower, new BrokerTopicStats, new MetadataCache(config.brokerId)) @@ -101,7 +100,7 @@ class ReplicaManagerTest { def testIllegalRequiredAcks() { val props = TestUtils.createBrokerConfig(1, TestUtils.MockZkConnect) val config = KafkaConfig.fromProps(props) - val mockLogMgr = TestUtils.createLogManager(config.logDirs.map(new File(_)).toArray) + val mockLogMgr = TestUtils.createLogManager(config.logDirs.map(new File(_)).toArray, zkUtils) val rm = new ReplicaManager(config, metrics, time, zkUtils, new MockScheduler(time), mockLogMgr, new AtomicBoolean(false), QuotaFactory.instantiate(config, metrics, time).follower, new BrokerTopicStats, new MetadataCache(config.brokerId), Option(this.getClass.getName)) @@ -130,7 +129,8 @@ class ReplicaManagerTest { props.put("log.dir", TestUtils.tempRelativeDir("data").getAbsolutePath) val config = KafkaConfig.fromProps(props) val logProps = new Properties() - val mockLogMgr = TestUtils.createLogManager(config.logDirs.map(new File(_)).toArray, LogConfig(logProps)) + logProps.put(LogConfig.MessageTimestampDifferenceMaxMsProp, Long.MaxValue.toString) + val mockLogMgr = TestUtils.createLogManager(config.logDirs.map(new File(_)).toArray, zkUtils, LogConfig(logProps)) val aliveBrokers = Seq(createBroker(0, "host0", 0), createBroker(1, "host1", 1)) val metadataCache = EasyMock.createMock(classOf[MetadataCache]) EasyMock.expect(metadataCache.getAliveBrokers).andReturn(aliveBrokers).anyTimes() @@ -146,7 +146,7 @@ class ReplicaManagerTest { partition.getOrCreateReplica(0) // Make this replica the leader. val leaderAndIsrRequest1 = new LeaderAndIsrRequest.Builder(0, 0, - collection.immutable.Map(new TopicPartition(topic, 0) -> new PartitionState(0, 0, 0, brokerList, 0, brokerList)).asJava, + collection.immutable.Map(new TopicPartition(topic, 0) -> new PartitionState(0, 0, 0, brokerList, 0, brokerList, false)).asJava, Set(new Node(0, "host1", 0), new Node(1, "host2", 1)).asJava).build() rm.becomeLeaderOrFollower(0, leaderAndIsrRequest1, (_, _) => ()) rm.getLeaderReplicaIfLocal(new TopicPartition(topic, 0)) @@ -163,7 +163,7 @@ class ReplicaManagerTest { // Make this replica the follower val leaderAndIsrRequest2 = new LeaderAndIsrRequest.Builder(0, 0, - collection.immutable.Map(new TopicPartition(topic, 0) -> new PartitionState(0, 1, 1, brokerList, 0, brokerList)).asJava, + collection.immutable.Map(new TopicPartition(topic, 0) -> new PartitionState(0, 1, 1, brokerList, 0, brokerList, false)).asJava, Set(new Node(0, "host1", 0), new Node(1, "host2", 1)).asJava).build() rm.becomeLeaderOrFollower(1, leaderAndIsrRequest2, (_, _) => ()) @@ -176,6 +176,20 @@ class ReplicaManagerTest { @Test def testReadCommittedFetchLimitedAtLSO(): Unit = { + val props = TestUtils.createBrokerConfig(1, TestUtils.MockZkConnect) + props.put("log.dir", TestUtils.tempRelativeDir("data").getAbsolutePath) + props.put("broker.id", Int.box(0)) + val config = KafkaConfig.fromProps(props) + val logProps = new Properties() + logProps.put(LogConfig.MessageTimestampDifferenceMaxMsProp, Long.MaxValue.toString) + val mockLogMgr = TestUtils.createLogManager(config.logDirs.map(new File(_)).toArray, zkUtils, LogConfig(logProps)) + val aliveBrokers = Seq(createBroker(0, "host0", 0), createBroker(1, "host1", 1)) + val metadataCache = EasyMock.createMock(classOf[MetadataCache]) + EasyMock.expect(metadataCache.getAliveBrokers).andReturn(aliveBrokers).anyTimes() + EasyMock.expect(metadataCache.isBrokerAlive(EasyMock.eq(0))).andReturn(true).anyTimes() + EasyMock.expect(metadataCache.isBrokerAlive(EasyMock.eq(1))).andReturn(true).anyTimes() + EasyMock.replay(metadataCache) + val timer = new MockTimer val replicaManager = setupReplicaManagerWithMockedPurgatories(timer) @@ -187,7 +201,7 @@ class ReplicaManagerTest { // Make this replica the leader. val leaderAndIsrRequest1 = new LeaderAndIsrRequest.Builder(0, 0, - collection.immutable.Map(new TopicPartition(topic, 0) -> new PartitionState(0, 0, 0, brokerList, 0, brokerList)).asJava, + collection.immutable.Map(new TopicPartition(topic, 0) -> new PartitionState(0, 0, 0, brokerList, 0, brokerList, true)).asJava, Set(new Node(0, "host1", 0), new Node(1, "host2", 1)).asJava).build() replicaManager.becomeLeaderOrFollower(0, leaderAndIsrRequest1, (_, _) => ()) replicaManager.getLeaderReplicaIfLocal(new TopicPartition(topic, 0)) @@ -271,14 +285,12 @@ class ReplicaManagerTest { try { val brokerList: java.util.List[Integer] = Seq[Integer](0, 1).asJava - val brokerSet: java.util.Set[Integer] = Set[Integer](0, 1).asJava - val partition = replicaManager.getOrCreatePartition(new TopicPartition(topic, 0)) partition.getOrCreateReplica(0) // Make this replica the leader. val leaderAndIsrRequest1 = new LeaderAndIsrRequest.Builder(0, 0, - collection.immutable.Map(new TopicPartition(topic, 0) -> new PartitionState(0, 0, 0, brokerList, 0, brokerList)).asJava, + collection.immutable.Map(new TopicPartition(topic, 0) -> new PartitionState(0, 0, 0, brokerList, 0, brokerList, true)).asJava, Set(new Node(0, "host1", 0), new Node(1, "host2", 1)).asJava).build() replicaManager.becomeLeaderOrFollower(0, leaderAndIsrRequest1, (_, _) => ()) replicaManager.getLeaderReplicaIfLocal(new TopicPartition(topic, 0)) @@ -336,7 +348,8 @@ class ReplicaManagerTest { props.put("broker.id", Int.box(0)) val config = KafkaConfig.fromProps(props) val logProps = new Properties() - val mockLogMgr = TestUtils.createLogManager(config.logDirs.map(new File(_)).toArray, LogConfig(logProps)) + logProps.put(LogConfig.MessageTimestampDifferenceMaxMsProp, Long.MaxValue.toString) + val mockLogMgr = TestUtils.createLogManager(config.logDirs.map(new File(_)).toArray, zkUtils, LogConfig(logProps)) val aliveBrokers = Seq(createBroker(0, "host0", 0), createBroker(1, "host1", 1), createBroker(1, "host2", 2)) val metadataCache = EasyMock.createMock(classOf[MetadataCache]) EasyMock.expect(metadataCache.getAliveBrokers).andReturn(aliveBrokers).anyTimes() @@ -356,7 +369,7 @@ class ReplicaManagerTest { // Make this replica the leader. val leaderAndIsrRequest1 = new LeaderAndIsrRequest.Builder(0, 0, - collection.immutable.Map(new TopicPartition(topic, 0) -> new PartitionState(0, 0, 0, brokerList, 0, brokerList)).asJava, + collection.immutable.Map(new TopicPartition(topic, 0) -> new PartitionState(0, 0, 0, brokerList, 0, brokerList, false)).asJava, Set(new Node(0, "host1", 0), new Node(1, "host2", 1), new Node(2, "host2", 2)).asJava).build() rm.becomeLeaderOrFollower(0, leaderAndIsrRequest1, (_, _) => ()) rm.getLeaderReplicaIfLocal(new TopicPartition(topic, 0)) @@ -481,7 +494,7 @@ class ReplicaManagerTest { props.put("broker.id", Int.box(0)) val config = KafkaConfig.fromProps(props) val logProps = new Properties() - val mockLogMgr = TestUtils.createLogManager(config.logDirs.map(new File(_)).toArray, LogConfig(logProps)) + val mockLogMgr = TestUtils.createLogManager(config.logDirs.map(new File(_)).toArray, zkUtils, LogConfig(logProps)) val aliveBrokers = Seq(createBroker(0, "host0", 0), createBroker(1, "host1", 1)) val metadataCache = EasyMock.createMock(classOf[MetadataCache]) EasyMock.expect(metadataCache.getAliveBrokers).andReturn(aliveBrokers).anyTimes() diff --git a/core/src/test/scala/unit/kafka/server/RequestQuotaTest.scala b/core/src/test/scala/unit/kafka/server/RequestQuotaTest.scala index 7c171b0dfe5f..c961c9b74c8b 100644 --- a/core/src/test/scala/unit/kafka/server/RequestQuotaTest.scala +++ b/core/src/test/scala/unit/kafka/server/RequestQuotaTest.scala @@ -181,14 +181,15 @@ class RequestQuotaTest extends BaseRequestTest { case ApiKeys.LEADER_AND_ISR => new LeaderAndIsrRequest.Builder(brokerId, Int.MaxValue, - Map(tp -> new PartitionState(Int.MaxValue, brokerId, Int.MaxValue, List(brokerId).asJava, 2, Seq(brokerId).asJava)).asJava, + Map(tp -> new PartitionState(Int.MaxValue, brokerId, Int.MaxValue, List(brokerId).asJava, 2, Seq(brokerId).asJava, true)).asJava, Set(new Node(brokerId, "localhost", 0)).asJava) case ApiKeys.STOP_REPLICA => new StopReplicaRequest.Builder(brokerId, Int.MaxValue, true, Set(tp).asJava) case ApiKeys.UPDATE_METADATA_KEY => - val partitionState = Map(tp -> new PartitionState(Int.MaxValue, brokerId, Int.MaxValue, List(brokerId).asJava, 2, Seq(brokerId).asJava)).asJava + val partitionState = Map(tp -> new UpdateMetadataRequest.UpdateMetadataRequestPartitionState( + Int.MaxValue, brokerId, Int.MaxValue, List(brokerId).asJava, 2, Seq(brokerId).asJava, Seq.empty[Integer].asJava)).asJava val securityProtocol = SecurityProtocol.PLAINTEXT val brokers = Set(new UpdateMetadataRequest.Broker(brokerId, Seq(new UpdateMetadataRequest.EndPoint("localhost", 0, securityProtocol, diff --git a/core/src/test/scala/unit/kafka/server/SimpleFetchTest.scala b/core/src/test/scala/unit/kafka/server/SimpleFetchTest.scala index 72d7fc5f40ef..3da3fa4056b3 100644 --- a/core/src/test/scala/unit/kafka/server/SimpleFetchTest.scala +++ b/core/src/test/scala/unit/kafka/server/SimpleFetchTest.scala @@ -16,6 +16,8 @@ */ package kafka.server +import java.io.File + import kafka.api._ import kafka.utils._ import kafka.cluster.Replica @@ -33,6 +35,8 @@ import org.apache.kafka.common.requests.IsolationLevel import org.easymock.EasyMock import org.junit.Assert._ +import scala.collection.mutable.ArrayBuffer + class SimpleFetchTest { val replicaLagTimeMaxMs = 100L @@ -105,6 +109,7 @@ class SimpleFetchTest { // create the log manager that is aware of this mock log val logManager = EasyMock.createMock(classOf[kafka.log.LogManager]) EasyMock.expect(logManager.getLog(topicPartition)).andReturn(Some(log)).anyTimes() + EasyMock.expect(logManager.liveLogDirs).andReturn(ArrayBuffer.empty[File]).anyTimes() EasyMock.replay(logManager) // create the replica manager diff --git a/core/src/test/scala/unit/kafka/server/epoch/OffsetsForLeaderEpochTest.scala b/core/src/test/scala/unit/kafka/server/epoch/OffsetsForLeaderEpochTest.scala index d004641a1b93..efc9b2868c3e 100644 --- a/core/src/test/scala/unit/kafka/server/epoch/OffsetsForLeaderEpochTest.scala +++ b/core/src/test/scala/unit/kafka/server/epoch/OffsetsForLeaderEpochTest.scala @@ -16,6 +16,7 @@ */ package kafka.server.epoch +import java.io.File import java.util.concurrent.atomic.AtomicBoolean import kafka.cluster.Replica @@ -30,6 +31,8 @@ import org.easymock.EasyMock._ import org.junit.Assert._ import org.junit.Test +import scala.collection.mutable.ArrayBuffer + class OffsetsForLeaderEpochTest { private val config = TestUtils.createBrokerConfigs(1, TestUtils.MockZkConnect).map(KafkaConfig.fromProps).head private val time = new MockTime @@ -46,12 +49,14 @@ class OffsetsForLeaderEpochTest { //Stubs val mockLog = createNiceMock(classOf[kafka.log.Log]) val mockCache = createNiceMock(classOf[kafka.server.epoch.LeaderEpochCache]) + val logManager = createNiceMock(classOf[kafka.log.LogManager]) expect(mockCache.endOffsetFor(epochRequested)).andReturn(offset) expect(mockLog.leaderEpochCache).andReturn(mockCache).anyTimes() - replay(mockCache, mockLog) + expect(logManager.liveLogDirs).andReturn(ArrayBuffer.empty[File]).anyTimes() + replay(mockCache, mockLog, logManager) // create a replica manager with 1 partition that has 1 replica - val replicaManager = new ReplicaManager(config, metrics, time, null, null, null, new AtomicBoolean(false), + val replicaManager = new ReplicaManager(config, metrics, time, null, null, logManager, new AtomicBoolean(false), QuotaFactory.instantiate(config, metrics, time).follower, new BrokerTopicStats, new MetadataCache(config.brokerId)) val partition = replicaManager.getOrCreatePartition(tp) @@ -68,8 +73,12 @@ class OffsetsForLeaderEpochTest { @Test def shouldReturnNoLeaderForPartitionIfThrown(): Unit = { + val logManager = createNiceMock(classOf[kafka.log.LogManager]) + expect(logManager.liveLogDirs).andReturn(ArrayBuffer.empty[File]).anyTimes() + replay(logManager) + //create a replica manager with 1 partition that has 0 replica - val replicaManager = new ReplicaManager(config, metrics, time, null, null, null, new AtomicBoolean(false), + val replicaManager = new ReplicaManager(config, metrics, time, null, null, logManager, new AtomicBoolean(false), QuotaFactory.instantiate(config, metrics, time).follower, new BrokerTopicStats, new MetadataCache(config.brokerId)) replicaManager.getOrCreatePartition(tp) @@ -87,8 +96,12 @@ class OffsetsForLeaderEpochTest { @Test def shouldReturnUnknownTopicOrPartitionIfThrown(): Unit = { + val logManager = createNiceMock(classOf[kafka.log.LogManager]) + expect(logManager.liveLogDirs).andReturn(ArrayBuffer.empty[File]).anyTimes() + replay(logManager) + //create a replica manager with 0 partition - val replicaManager = new ReplicaManager(config, metrics, time, null, null, null, new AtomicBoolean(false), + val replicaManager = new ReplicaManager(config, metrics, time, null, null, logManager, new AtomicBoolean(false), QuotaFactory.instantiate(config, metrics, time).follower, new BrokerTopicStats, new MetadataCache(config.brokerId)) diff --git a/core/src/test/scala/unit/kafka/utils/TestUtils.scala b/core/src/test/scala/unit/kafka/utils/TestUtils.scala index 14f0114bfd8e..d9274be0cc5e 100755 --- a/core/src/test/scala/unit/kafka/utils/TestUtils.scala +++ b/core/src/test/scala/unit/kafka/utils/TestUtils.scala @@ -1006,9 +1006,11 @@ object TestUtils extends Logging { * Create new LogManager instance with default configuration for testing */ def createLogManager(logDirs: Array[File] = Array.empty[File], + zkUtils: ZkUtils, defaultConfig: LogConfig = LogConfig(), cleanerConfig: CleanerConfig = CleanerConfig(enableCleaner = false), - time: MockTime = new MockTime()): LogManager = { + time: MockTime = new MockTime(), + brokerId: Int = 0): LogManager = { new LogManager(logDirs = logDirs, topicConfigs = Map(), defaultConfig = defaultConfig, @@ -1021,8 +1023,10 @@ object TestUtils extends Logging { maxPidExpirationMs = 60 * 60 * 1000, scheduler = time.scheduler, time = time, - brokerState = BrokerState(), - brokerTopicStats = new BrokerTopicStats) + brokerState = new BrokerState(), + brokerTopicStats = new BrokerTopicStats, + zkUtils = zkUtils, + brokerId = brokerId) } @deprecated("This method has been deprecated and it will be removed in a future release.", "0.10.0.0") @@ -1161,7 +1165,7 @@ object TestUtils extends Logging { servers.forall(server => topicPartitions.forall(tp => server.getLogManager().getLog(tp).isEmpty))) // ensure that topic is removed from all cleaner offsets TestUtils.waitUntilTrue(() => servers.forall(server => topicPartitions.forall { tp => - val checkpoints = server.getLogManager().logDirs.map { logDir => + val checkpoints = server.getLogManager().liveLogDirs.map { logDir => new OffsetCheckpointFile(new File(logDir, "cleaner-offset-checkpoint")).read() } checkpoints.forall(checkpointsPerLogDir => !checkpointsPerLogDir.contains(tp)) diff --git a/streams/src/test/java/org/apache/kafka/streams/integration/utils/IntegrationTestUtils.java b/streams/src/test/java/org/apache/kafka/streams/integration/utils/IntegrationTestUtils.java index 7cdc180b1c5b..eb844d3e8e3a 100644 --- a/streams/src/test/java/org/apache/kafka/streams/integration/utils/IntegrationTestUtils.java +++ b/streams/src/test/java/org/apache/kafka/streams/integration/utils/IntegrationTestUtils.java @@ -16,7 +16,7 @@ */ package org.apache.kafka.streams.integration.utils; -import kafka.api.PartitionStateInfo; +import kafka.api.MetadataPartitionState; import kafka.api.Request; import kafka.server.KafkaServer; import kafka.server.MetadataCache; @@ -225,13 +225,13 @@ public static void waitUntilMetadataIsPropagated(final List servers public boolean conditionMet() { for (final KafkaServer server : servers) { final MetadataCache metadataCache = server.apis().metadataCache(); - final Option partitionInfo = + final Option partitionInfo = metadataCache.getPartitionInfo(topic, partition); if (partitionInfo.isEmpty()) { return false; } - final PartitionStateInfo partitionStateInfo = partitionInfo.get(); - if (!Request.isValidBrokerId(partitionStateInfo.leaderIsrAndControllerEpoch().leaderAndIsr().leader())) { + final MetadataPartitionState metadataPartitionState = partitionInfo.get(); + if (!Request.isValidBrokerId(metadataPartitionState.leaderIsrAndControllerEpoch().leaderAndIsr().leader())) { return false; } } diff --git a/streams/src/test/java/org/apache/kafka/streams/processor/internals/InternalTopicManagerTest.java b/streams/src/test/java/org/apache/kafka/streams/processor/internals/InternalTopicManagerTest.java index aef0d351c967..d7a99b8c3d6d 100644 --- a/streams/src/test/java/org/apache/kafka/streams/processor/internals/InternalTopicManagerTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/processor/internals/InternalTopicManagerTest.java @@ -132,7 +132,7 @@ public void createTopics(final Map topicsMap, fina @Override public MetadataResponse fetchMetadata() { Node node = new Node(1, "host1", 1001); - MetadataResponse.PartitionMetadata partitionMetadata = new MetadataResponse.PartitionMetadata(Errors.NONE, 1, node, new ArrayList(), new ArrayList()); + MetadataResponse.PartitionMetadata partitionMetadata = new MetadataResponse.PartitionMetadata(Errors.NONE, 1, node, new ArrayList(), new ArrayList(), new ArrayList()); MetadataResponse.TopicMetadata topicMetadata = new MetadataResponse.TopicMetadata(Errors.NONE, topic, true, Collections.singletonList(partitionMetadata)); MetadataResponse response = new MetadataResponse(Collections.singletonList(node), null, MetadataResponse.NO_CONTROLLER_ID, Collections.singletonList(topicMetadata)); diff --git a/tests/kafkatest/services/kafka/config.py b/tests/kafkatest/services/kafka/config.py index 1b81b43e5045..462277f06312 100644 --- a/tests/kafkatest/services/kafka/config.py +++ b/tests/kafkatest/services/kafka/config.py @@ -24,7 +24,7 @@ class KafkaConfig(dict): DEFAULTS = { config_property.PORT: 9092, config_property.SOCKET_RECEIVE_BUFFER_BYTES: 65536, - config_property.LOG_DIRS: "/mnt/kafka-data-logs", + config_property.LOG_DIRS: "/mnt/kafka-data-logs-1,/mnt/kafka-data-logs-2", config_property.ZOOKEEPER_CONNECTION_TIMEOUT_MS: 2000 } diff --git a/tests/kafkatest/services/kafka/config_property.py b/tests/kafkatest/services/kafka/config_property.py index a9ba40d79c43..8b64d0e96005 100644 --- a/tests/kafkatest/services/kafka/config_property.py +++ b/tests/kafkatest/services/kafka/config_property.py @@ -44,6 +44,10 @@ MESSAGE_TIMESTAMP_TYPE = "message.timestamp.type" THROTTLING_REPLICATION_RATE_LIMIT = "replication.quota.throttled.rate" +LOG_FLUSH_INTERVAL_MESSAGE = "log.flush.interval.messages" +REPLICA_HIGHWATERMARK_CHECKPOINT_INTERVAL_MS = "replica.high.watermark.checkpoint.interval.ms" +LOG_ROLL_TIME_MS = "log.roll.ms" + """ From KafkaConfig.scala diff --git a/tests/kafkatest/services/kafka/kafka.py b/tests/kafkatest/services/kafka/kafka.py index 7e4a5899e6c4..08f724c711f1 100644 --- a/tests/kafkatest/services/kafka/kafka.py +++ b/tests/kafkatest/services/kafka/kafka.py @@ -45,7 +45,9 @@ class KafkaService(KafkaPathResolverMixin, JmxMixin, Service): OPERATIONAL_LOG_INFO_DIR = os.path.join(OPERATIONAL_LOG_DIR, "info") OPERATIONAL_LOG_DEBUG_DIR = os.path.join(OPERATIONAL_LOG_DIR, "debug") # Kafka log segments etc go here - DATA_LOG_DIR = os.path.join(PERSISTENT_ROOT, "kafka-data-logs") + DATA_LOG_DIR_PREFIX = os.path.join(PERSISTENT_ROOT, "kafka-data-logs") + DATA_LOG_DIR_1 = "%s-1" % (DATA_LOG_DIR_PREFIX) + DATA_LOG_DIR_2 = "%s-2" % (DATA_LOG_DIR_PREFIX) CONFIG_FILE = os.path.join(PERSISTENT_ROOT, "kafka.properties") # Kafka Authorizer SIMPLE_AUTHORIZER = "kafka.security.auth.SimpleAclAuthorizer" @@ -60,15 +62,18 @@ class KafkaService(KafkaPathResolverMixin, JmxMixin, Service): "kafka_operational_logs_debug": { "path": OPERATIONAL_LOG_DEBUG_DIR, "collect_default": False}, - "kafka_data": { - "path": DATA_LOG_DIR, + "kafka_data_1": { + "path": DATA_LOG_DIR_1, + "collect_default": False}, + "kafka_data_2": { + "path": DATA_LOG_DIR_2, "collect_default": False} } def __init__(self, context, num_nodes, zk, security_protocol=SecurityConfig.PLAINTEXT, interbroker_security_protocol=SecurityConfig.PLAINTEXT, client_sasl_mechanism=SecurityConfig.SASL_MECHANISM_GSSAPI, interbroker_sasl_mechanism=SecurityConfig.SASL_MECHANISM_GSSAPI, authorizer_class_name=None, topics=None, version=DEV_BRANCH, jmx_object_names=None, - jmx_attributes=None, zk_connect_timeout=5000, zk_session_timeout=6000, server_prop_overides=[]): + jmx_attributes=None, zk_connect_timeout=5000, zk_session_timeout=6000, server_prop_overides={}): """ :type context :type zk: ZookeeperService @@ -149,6 +154,9 @@ def start_minikdc(self, add_principals=""): else: self.minikdc = None + def alive(self, node): + return len(self.pids(node)) > 0 + def start(self, add_principals=""): self.open_port(self.security_protocol) self.open_port(self.interbroker_security_protocol) @@ -183,8 +191,8 @@ def prop_file(self, node): cfg[config_property.ADVERTISED_HOSTNAME] = node.account.hostname cfg[config_property.ZOOKEEPER_CONNECT] = self.zk.connect_setting() - for prop in self.server_prop_overides: - cfg[prop[0]] = prop[1] + for key, value in self.server_prop_overides.iteritems(): + cfg[key] = value self.set_protocol_and_port(node) @@ -261,7 +269,7 @@ def clean_node(self, node): JmxMixin.clean_node(self, node) self.security_config.clean_node(node) node.account.kill_process("kafka", clean_shutdown=False, allow_fail=True) - node.account.ssh("rm -rf /mnt/*", allow_fail=False) + node.account.ssh("sudo rm -rf /mnt/*", allow_fail=False) def create_topic(self, topic_cfg, node=None): """Run the admin tool create topic command. @@ -443,7 +451,7 @@ def search_data_files(self, topic, messages): len(messages)) for node in self.nodes: # Grab all .log files in directories prefixed with this topic - files = node.account.ssh_capture("find %s -regex '.*/%s-.*/[^/]*.log'" % (KafkaService.DATA_LOG_DIR, topic)) + files = node.account.ssh_capture("find %s* -regex '.*/%s-.*/[^/]*.log'" % (KafkaService.DATA_LOG_DIR_PREFIX, topic)) # Check each data file to see if it contains the messages we want for log in files: @@ -470,10 +478,45 @@ def restart_node(self, node, clean_shutdown=True): self.stop_node(node, clean_shutdown) self.start_node(node) + def isr_idx_list(self, topic, partition=0): + """ Get the leader replica id for the given topic and partition. + """ + self.logger.debug("Querying zookeeper to find in-sync replicas for topic %s and partition %d" % (topic, partition)) + zk_path = "/brokers/topics/%s/partitions/%d/state" % (topic, partition) + partition_state = self.zk.query(zk_path) + + if partition_state is None: + raise Exception("Error finding partition state for topic %s and partition %d." % (topic, partition)) + + partition_state = json.loads(partition_state) + self.logger.info(partition_state) + + isr_idx_list = partition_state["isr"] + self.logger.info("Leader for topic %s and partition %d is now: %s" % (topic, partition, isr_idx_list)) + return isr_idx_list + + def replicas(self, topic, partition=0): + """ Get the assigned replicas for the given topic and partition. + """ + self.logger.debug("Querying zookeeper to find assigned replicas for topic %s and partition %d" % (topic, partition)) + zk_path = "/brokers/topics/%s" % (topic) + assignemnt = self.zk.query(zk_path) + + if assignemnt is None: + raise Exception("Error finding partition state for topic %s and partition %d." % (topic, partition)) + + assignemnt = json.loads(assignemnt) + self.logger.info(assignemnt) + + replicas = assignemnt["partitions"][str(partition)] + + self.logger.info("Assigned replicas for topic %s and partition %d is now: %s" % (topic, partition, replicas)) + return [self.get_node(replica) for replica in replicas] + def leader(self, topic, partition=0): """ Get the leader replica for the given topic and partition. """ - self.logger.debug("Querying zookeeper to find leader replica for topic: \n%s" % (topic)) + self.logger.debug("Querying zookeeper to find leader replica for topic %s and partition %d" % (topic, partition)) zk_path = "/brokers/topics/%s/partitions/%d/state" % (topic, partition) partition_state = self.zk.query(zk_path) diff --git a/tests/kafkatest/tests/core/log_dir_failure_test.py b/tests/kafkatest/tests/core/log_dir_failure_test.py new file mode 100644 index 000000000000..492ce61ce645 --- /dev/null +++ b/tests/kafkatest/tests/core/log_dir_failure_test.py @@ -0,0 +1,177 @@ +# 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. + +import os.path + +from ducktape.utils.util import wait_until +from ducktape.mark import matrix +from ducktape.mark import parametrize +from ducktape.mark.resource import cluster +from kafkatest.services.kafka import config_property +from kafkatest.services.zookeeper import ZookeeperService +from kafkatest.services.kafka import KafkaService +from kafkatest.services.verifiable_producer import VerifiableProducer +from kafkatest.services.console_consumer import ConsoleConsumer +from kafkatest.tests.produce_consume_validate import ProduceConsumeValidateTest +from kafkatest.utils import is_int + +import signal + + +def select_node(test, broker_type, topic): + """ Discover node of requested type. For leader type, discovers leader for our topic and partition 0 + """ + if broker_type == "leader": + node = test.kafka.leader(topic, partition=0) + elif broker_type == "follower": + leader = test.kafka.leader(topic, partition=0) + node = [replica for replica in test.kafka.replicas(topic, partition=0) if replica != leader][0] + elif broker_type == "controller": + node = test.kafka.controller() + else: + raise Exception("Unexpected broker type %s." % (broker_type)) + + return node + + +class LogDirFailureTest(ProduceConsumeValidateTest): + """ + Note that consuming is a bit tricky, at least with console consumer. The goal is to consume all messages + (foreach partition) in the topic. In this case, waiting for the last message may cause the consumer to stop + too soon since console consumer is consuming multiple partitions from a single thread and therefore we lose + ordering guarantees. + + Waiting on a count of consumed messages can be unreliable: if we stop consuming when num_consumed == num_acked, + we might exit early if some messages are duplicated (though not an issue here since producer retries==0) + + Therefore rely here on the consumer.timeout.ms setting which times out on the interval between successively + consumed messages. Since we run the producer to completion before running the consumer, this is a reliable + indicator that nothing is left to consume. + """ + + def __init__(self, test_context): + """:type test_context: ducktape.tests.test.TestContext""" + super(LogDirFailureTest, self).__init__(test_context=test_context) + + self.topic1 = "test_topic_1" + self.topic2 = "test_topic_2" + self.zk = ZookeeperService(test_context, num_nodes=1) + self.kafka = KafkaService(test_context, + num_nodes=3, + zk=self.zk, + topics={ + self.topic1: {"partitions": 1, "replication-factor": 3, "configs": {"min.insync.replicas": 2}}, + self.topic2: {"partitions": 1, "replication-factor": 3, "configs": {"min.insync.replicas": 1}} + }, + server_prop_overides={ + config_property.LOG_FLUSH_INTERVAL_MESSAGE: 5, + config_property.REPLICA_HIGHWATERMARK_CHECKPOINT_INTERVAL_MS: 60000, + config_property.LOG_ROLL_TIME_MS: 3000 + }) + + self.producer_throughput = 1000 + self.num_producers = 1 + self.num_consumers = 1 + + def setUp(self): + self.zk.start() + + def min_cluster_size(self): + """Override this since we're adding services outside of the constructor""" + return super(LogDirFailureTest, self).min_cluster_size() + self.num_producers * 2 + self.num_consumers * 2 + + @cluster(num_nodes=9) + @matrix(bounce_broker=[False, True], broker_type=["leader", "follower"], security_protocol=["PLAINTEXT"]) + def test_replication_with_disk_failure(self, bounce_broker, security_protocol, broker_type): + """Replication tests. + These tests verify that replication provides simple durability guarantees by checking that data acked by + brokers is still available for consumption in the face of various failure scenarios. + + Setup: 1 zk, 3 kafka nodes, 1 topic with partitions=3, replication-factor=3, and min.insync.replicas=2 + + - Produce messages in the background + - Consume messages in the background + - Drive broker failures (shutdown, or bounce repeatedly with kill -15 or kill -9) + - When done driving failures, stop producing, and finish consuming + - Validate that every acked message was consumed + """ + + self.kafka.security_protocol = security_protocol + self.kafka.interbroker_security_protocol = security_protocol + self.kafka.start() + + try: + # Initialize producer/consumer for topic1 + self.producer = VerifiableProducer(self.test_context, self.num_producers, self.kafka, self.topic1, + throughput=self.producer_throughput) + self.consumer = ConsoleConsumer(self.test_context, self.num_consumers, self.kafka, self.topic1, group_id="test-consumer-group-1", + new_consumer=False, consumer_timeout_ms=60000, message_validator=is_int) + self.start_producer_and_consumer() + + # Get a replica of the partition of topic1 and make its first log directory offline by changing the log dir's permission. + # We assume that partition of topic1 is created in the first log directory of respective brokers. + broker_node = select_node(self, broker_type, self.topic1) + broker_idx = self.kafka.idx(broker_node) + assert broker_idx in self.kafka.isr_idx_list(self.topic1), \ + "Leader node %d should be in isr set %s" % (broker_idx, str(self.kafka.isr_idx_list(self.topic1))) + + self.logger.debug("Making log dir %s inaccessible" % (KafkaService.DATA_LOG_DIR_1)) + cmd = "sudo chmod a-rw %s -R" % (KafkaService.DATA_LOG_DIR_1) + broker_node.account.ssh(cmd, allow_fail=False) + + if bounce_broker: + self.kafka.restart_node(broker_node, clean_shutdown=True) + + # Verify the following: + # 1) The broker with offline log directory is not the leader of the partition of topic1 + # 2) The broker with offline log directory is not in the ISR + # 3) The broker with offline log directory is still online + # 4) Messages can still be produced and consumed from topic1 + wait_until(lambda: self.kafka.leader(self.topic1, partition=0) != broker_node, + timeout_sec=60, + err_msg="Broker %d should not be leader of topic %s and partition 0" % (broker_idx, self.topic1)) + assert self.kafka.alive(broker_node), "Broker %d should be still online" % (broker_idx) + wait_until(lambda: broker_idx not in self.kafka.isr_idx_list(self.topic1), + timeout_sec=60, + err_msg="Broker %d should not be in isr set %s" % (broker_idx, str(self.kafka.isr_idx_list(self.topic1)))) + + self.stop_producer_and_consumer() + self.validate() + + # Shutdown all other brokers so that the broker with offline log dir is the only online broker + for node in self.kafka.nodes: + if broker_node != node: + self.logger.debug("Hard shutdown broker %d" % (self.kafka.idx(node))) + self.kafka.signal_node(node, sig=signal.SIGKILL) + + # Verify the following: + # 1) The broker with offline directory is the only in-sync broker of the partition of topic2 + # 2) Messages can still be produced and consumed from topic2 + self.producer = VerifiableProducer(self.test_context, self.num_producers, self.kafka, self.topic2, + throughput=self.producer_throughput) + self.consumer = ConsoleConsumer(self.test_context, self.num_consumers, self.kafka, self.topic2, group_id="test-consumer-group-2", + new_consumer=False, consumer_timeout_ms=60000, message_validator=is_int) + self.start_producer_and_consumer() + + assert self.kafka.isr_idx_list(self.topic2) == [broker_idx], \ + "In-sync replicas of topic %s and partition 0 should be %s" % (self.topic2, str([broker_idx])) + + self.stop_producer_and_consumer() + self.validate() + + except BaseException as e: + for s in self.test_context.services: + self.mark_for_collect(s) + raise From 0c97e0907e93c00fc23958478ec103f6ac49a48d Mon Sep 17 00:00:00 2001 From: Dong Lin Date: Sun, 11 Jun 2017 21:05:12 -0700 Subject: [PATCH 02/11] Address comments --- .../apache/kafka/common/PartitionInfo.java | 3 +- .../kafka/common/protocol/Protocol.java | 6 +- .../common/requests/LeaderAndIsrRequest.java | 5 +- .../common/requests/MetadataResponse.java | 2 +- .../kafka/common/requests/PartitionState.java | 2 +- .../requests/UpdateMetadataRequest.java | 1 - .../common/requests/RequestResponseTest.java | 4 +- .../src/main/scala/kafka/api/ApiVersion.scala | 11 ++- .../controller/ControllerChannelManager.scala | 25 +++-- .../kafka/controller/KafkaController.scala | 92 ++++++++----------- .../controller/PartitionStateMachine.scala | 4 +- .../controller/ReplicaStateMachine.scala | 4 +- .../src/main/scala/kafka/log/LogManager.scala | 69 ++++++++------ .../scala/kafka/server/ReplicaManager.scala | 3 +- .../kafka/api/AuthorizerIntegrationTest.scala | 2 +- .../kafka/server/LeaderElectionTest.scala | 3 +- .../kafka/server/ReplicaManagerTest.scala | 12 +-- .../unit/kafka/server/RequestQuotaTest.scala | 2 +- 18 files changed, 128 insertions(+), 122 deletions(-) diff --git a/clients/src/main/java/org/apache/kafka/common/PartitionInfo.java b/clients/src/main/java/org/apache/kafka/common/PartitionInfo.java index b101a9c87ede..0d979d7fcccb 100644 --- a/clients/src/main/java/org/apache/kafka/common/PartitionInfo.java +++ b/clients/src/main/java/org/apache/kafka/common/PartitionInfo.java @@ -17,9 +17,8 @@ package org.apache.kafka.common; /** - * Information about a topic-partition. + * Information about a topic-partition. This is used to describe MetadataPartitionInfo. */ -// TODO: rename this class to MetadataPartitionInfo public class PartitionInfo { private final String topic; diff --git a/clients/src/main/java/org/apache/kafka/common/protocol/Protocol.java b/clients/src/main/java/org/apache/kafka/common/protocol/Protocol.java index 967cf083b77f..c2775369acd5 100644 --- a/clients/src/main/java/org/apache/kafka/common/protocol/Protocol.java +++ b/clients/src/main/java/org/apache/kafka/common/protocol/Protocol.java @@ -1087,15 +1087,13 @@ public class Protocol { new Field("isr", new ArrayOf(INT32), "The in sync replica ids."), new Field("zk_version", INT32, "The ZK version."), new Field("replicas", new ArrayOf(INT32), "The replica ids."), - new Field("is_new", BOOLEAN, "If this is false, the replica will only be created if there is no bad log directory")); + new Field("is_new", BOOLEAN, "Whether the replica should have existed on the broker or not")); public static final Schema LEADER_AND_ISR_REQUEST_LIVE_LEADER_V0 = new Schema(new Field("id", INT32, "The broker id."), new Field("host", STRING, "The hostname of the broker."), new Field("port", INT32, "The port on which the broker accepts requests.")); - public static final Schema LEADER_AND_ISR_REQUEST_LIVE_LEADER_V1 = LEADER_AND_ISR_REQUEST_LIVE_LEADER_V0; - public static final Schema LEADER_AND_ISR_REQUEST_V0 = new Schema(new Field("controller_id", INT32, "The controller id."), new Field("controller_epoch", INT32, "The controller epoch."), new Field("partition_states", @@ -1106,7 +1104,7 @@ public class Protocol { new Field("controller_epoch", INT32, "The controller epoch."), new Field("partition_states", new ArrayOf(LEADER_AND_ISR_REQUEST_PARTITION_STATE_V1)), - new Field("live_leaders", new ArrayOf(LEADER_AND_ISR_REQUEST_LIVE_LEADER_V1))); + new Field("live_leaders", new ArrayOf(LEADER_AND_ISR_REQUEST_LIVE_LEADER_V0))); public static final Schema LEADER_AND_ISR_RESPONSE_PARTITION_V0 = new Schema(new Field("topic", STRING, "Topic name."), new Field("partition", INT32, "Topic partition id."), diff --git a/clients/src/main/java/org/apache/kafka/common/requests/LeaderAndIsrRequest.java b/clients/src/main/java/org/apache/kafka/common/requests/LeaderAndIsrRequest.java index 63ce76d2a4a0..733c9af39da6 100644 --- a/clients/src/main/java/org/apache/kafka/common/requests/LeaderAndIsrRequest.java +++ b/clients/src/main/java/org/apache/kafka/common/requests/LeaderAndIsrRequest.java @@ -58,9 +58,9 @@ public static class Builder extends AbstractRequest.Builder private final Map partitionStates; private final Set liveLeaders; - public Builder(int controllerId, int controllerEpoch, + public Builder(short version, int controllerId, int controllerEpoch, Map partitionStates, Set liveLeaders) { - super(ApiKeys.LEADER_AND_ISR); + super(ApiKeys.LEADER_AND_ISR, version); this.controllerId = controllerId; this.controllerEpoch = controllerEpoch; this.partitionStates = partitionStates; @@ -126,7 +126,6 @@ public LeaderAndIsrRequest(Struct struct, short version) { PartitionState partitionState = new PartitionState(controllerEpoch, leader, leaderEpoch, isr, zkVersion, replicas, isNew); partitionStates.put(new TopicPartition(topic, partition), partitionState); - } Set leaders = new HashSet<>(); 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 678ec296aa90..66559340bf69 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 @@ -338,7 +338,7 @@ public List partitionMetadata() { } - // TODO: rename this class to MetadataResponsePartitionState + // This is used to describe MetadataResponsePartitionState public static class PartitionMetadata { private final Errors error; private final int partition; diff --git a/clients/src/main/java/org/apache/kafka/common/requests/PartitionState.java b/clients/src/main/java/org/apache/kafka/common/requests/PartitionState.java index 41bd2e123487..8ca7c1f638d7 100644 --- a/clients/src/main/java/org/apache/kafka/common/requests/PartitionState.java +++ b/clients/src/main/java/org/apache/kafka/common/requests/PartitionState.java @@ -20,7 +20,7 @@ import java.util.List; -// TODO: rename this class to LeaderAndIsrPartitionInfo +// This is used to describe LeaderAndIsrPartitionInfo public class PartitionState { public final int controllerEpoch; public final int leader; diff --git a/clients/src/main/java/org/apache/kafka/common/requests/UpdateMetadataRequest.java b/clients/src/main/java/org/apache/kafka/common/requests/UpdateMetadataRequest.java index 197d35083f0b..65d4d788fa6e 100644 --- a/clients/src/main/java/org/apache/kafka/common/requests/UpdateMetadataRequest.java +++ b/clients/src/main/java/org/apache/kafka/common/requests/UpdateMetadataRequest.java @@ -75,7 +75,6 @@ public String toString() { } } - // TODO: rename this class to UpdateMetadataPartitionState and probably move it to its own class public static final class UpdateMetadataRequestPartitionState { public final int controllerEpoch; public final int leader; diff --git a/clients/src/test/java/org/apache/kafka/common/requests/RequestResponseTest.java b/clients/src/test/java/org/apache/kafka/common/requests/RequestResponseTest.java index f113e8b2e3e9..e5adc83a01a7 100644 --- a/clients/src/test/java/org/apache/kafka/common/requests/RequestResponseTest.java +++ b/clients/src/test/java/org/apache/kafka/common/requests/RequestResponseTest.java @@ -818,8 +818,8 @@ private LeaderAndIsrRequest createLeaderAndIsrRequest() { new Node(0, "test0", 1223), new Node(1, "test1", 1223) ); - - return new LeaderAndIsrRequest.Builder(1, 10, partitionStates, leaders).build(); + short version = ApiKeys.LEADER_AND_ISR.latestVersion(); + return new LeaderAndIsrRequest.Builder(version, 1, 10, partitionStates, leaders).build(); } private LeaderAndIsrResponse createLeaderAndIsrResponse() { diff --git a/core/src/main/scala/kafka/api/ApiVersion.scala b/core/src/main/scala/kafka/api/ApiVersion.scala index 62d5d1261ccb..f5d9d2060572 100644 --- a/core/src/main/scala/kafka/api/ApiVersion.scala +++ b/core/src/main/scala/kafka/api/ApiVersion.scala @@ -69,7 +69,9 @@ object ApiVersion { "0.11.0-IV1" -> KAFKA_0_11_0_IV1, // Introduced leader epoch fetches to the replica fetcher via KIP-101 "0.11.0-IV2" -> KAFKA_0_11_0_IV2, - "0.11.0" -> KAFKA_0_11_0_IV2 + "0.11.0" -> KAFKA_0_11_0_IV2, + // Introduced LeaderAndIsrRequest V1, UpdateMetadataRequest V4 and MetadataResponse V5 via KIP-112 + "0.11.1-IV0" -> KAFKA_0_11_1_IV0 ) private val versionPattern = "\\.".r @@ -171,3 +173,10 @@ case object KAFKA_0_11_0_IV2 extends ApiVersion { val messageFormatVersion: Byte = RecordBatch.MAGIC_VALUE_V2 val id: Int = 12 } + +case object KAFKA_0_11_1_IV0 extends ApiVersion { + val version: String = "0.11.1-IV0" + val messageFormatVersion: Byte = RecordBatch.MAGIC_VALUE_V2 + val id: Int = 13 +} + diff --git a/core/src/main/scala/kafka/controller/ControllerChannelManager.scala b/core/src/main/scala/kafka/controller/ControllerChannelManager.scala index 27927f5d24d4..42fdd30400db 100755 --- a/core/src/main/scala/kafka/controller/ControllerChannelManager.scala +++ b/core/src/main/scala/kafka/controller/ControllerChannelManager.scala @@ -278,7 +278,7 @@ class RequestSendThread(val controllerId: Int, } -class ControllerBrokerRequestBatch(controller: KafkaController, eventManager: ControllerEventManager) extends Logging { +class ControllerBrokerRequestBatch(controller: KafkaController) extends Logging { val controllerContext = controller.controllerContext val controllerId: Int = controller.config.brokerId val leaderAndIsrRequestMap = mutable.Map.empty[Int, mutable.Map[TopicPartition, LeaderAndIsrPartitionState]] @@ -316,7 +316,7 @@ class ControllerBrokerRequestBatch(controller: KafkaController, eventManager: Co brokerIds.filter(_ >= 0).foreach { brokerId => val result = leaderAndIsrRequestMap.getOrElseUpdate(brokerId, mutable.Map.empty) - val alreadyNew = result.get(topicPartition).map(_.isNew).getOrElse(false) + val alreadyNew = result.get(topicPartition).exists(_.isNew) result.put(topicPartition, LeaderAndIsrPartitionState(leaderIsrAndControllerEpoch, replicas, isNew || alreadyNew)) } @@ -381,6 +381,10 @@ class ControllerBrokerRequestBatch(controller: KafkaController, eventManager: Co def sendRequestsToBrokers(controllerEpoch: Int) { try { + val leaderAndIsrRequestVersion: Short = + if (controller.config.interBrokerProtocolVersion >= KAFKA_0_11_1_IV0) 1 + else 0 + leaderAndIsrRequestMap.foreach { case (broker, leaderAndIsrPartitionStates) => leaderAndIsrPartitionStates.foreach { case (topicPartition, state) => val typeOfRequest = @@ -402,10 +406,10 @@ class ControllerBrokerRequestBatch(controller: KafkaController, eventManager: Co leaderAndIsrPartitionState.allReplicas.map(Integer.valueOf).asJava, leaderAndIsrPartitionState.isNew) topicPartition -> partitionState } - val leaderAndIsrRequest = new LeaderAndIsrRequest.Builder(controllerId, controllerEpoch, partitionStates.asJava, - leaders.asJava) - controller.sendRequest(broker, ApiKeys.LEADER_AND_ISR, leaderAndIsrRequest, - (r: AbstractResponse) => eventManager.put(controller.LeaderAndIsrResponseReceived(r, broker))) + val leaderAndIsrRequestBuilder = new LeaderAndIsrRequest.Builder(leaderAndIsrRequestVersion, controllerId, + controllerEpoch, partitionStates.asJava, leaders.asJava) + controller.sendRequest(broker, ApiKeys.LEADER_AND_ISR, leaderAndIsrRequestBuilder, + (r: AbstractResponse) => controller.eventManager.put(controller.LeaderAndIsrResponseReceived(r, broker))) } leaderAndIsrRequestMap.clear() @@ -420,14 +424,15 @@ class ControllerBrokerRequestBatch(controller: KafkaController, eventManager: Co topicPartition -> partitionState } - val version: Short = - if (controller.config.interBrokerProtocolVersion >= KAFKA_0_10_2_IV0) 3 + val updateMetadataRequestVersion: Short = + if (controller.config.interBrokerProtocolVersion >= KAFKA_0_11_1_IV0) 4 + else if (controller.config.interBrokerProtocolVersion >= KAFKA_0_10_2_IV0) 3 else if (controller.config.interBrokerProtocolVersion >= KAFKA_0_10_0_IV1) 2 else if (controller.config.interBrokerProtocolVersion >= KAFKA_0_9_0) 1 else 0 val updateMetadataRequest = { - val liveBrokers = if (version == 0) { + val liveBrokers = if (updateMetadataRequestVersion == 0) { // Version 0 of UpdateMetadataRequest only supports PLAINTEXT. controllerContext.liveOrShuttingDownBrokers.map { broker => val securityProtocol = SecurityProtocol.PLAINTEXT @@ -444,7 +449,7 @@ class ControllerBrokerRequestBatch(controller: KafkaController, eventManager: Co new UpdateMetadataRequest.Broker(broker.id, endPoints.asJava, broker.rack.orNull) } } - new UpdateMetadataRequest.Builder(version, controllerId, controllerEpoch, partitionStates.asJava, + new UpdateMetadataRequest.Builder(updateMetadataRequestVersion, controllerId, controllerEpoch, partitionStates.asJava, liveBrokers.asJava) } diff --git a/core/src/main/scala/kafka/controller/KafkaController.scala b/core/src/main/scala/kafka/controller/KafkaController.scala index 9d37917656e7..1e63b6ee9fa3 100644 --- a/core/src/main/scala/kafka/controller/KafkaController.scala +++ b/core/src/main/scala/kafka/controller/KafkaController.scala @@ -167,8 +167,8 @@ class KafkaController(val config: KafkaConfig, zkUtils: ZkUtils, time: Time, met private[controller] val eventManager = new ControllerEventManager(controllerContext.stats.rateAndTimeMetrics, _ => updateMetrics()) - val partitionStateMachine = new PartitionStateMachine(this, eventManager) - val replicaStateMachine = new ReplicaStateMachine(this, eventManager) + val partitionStateMachine = new PartitionStateMachine(this) + val replicaStateMachine = new ReplicaStateMachine(this) // have a separate scheduler for the controller to be able to start and stop independently of the kafka server // visible for testing @@ -180,7 +180,7 @@ class KafkaController(val config: KafkaConfig, zkUtils: ZkUtils, time: Time, met private val reassignedPartitionLeaderSelector = new ReassignedPartitionLeaderSelector(controllerContext) private val preferredReplicaPartitionLeaderSelector = new PreferredReplicaPartitionLeaderSelector(controllerContext) private val controlledShutdownPartitionLeaderSelector = new ControlledShutdownLeaderSelector(controllerContext) - private val brokerRequestBatch = new ControllerBrokerRequestBatch(this, eventManager) + private val brokerRequestBatch = new ControllerBrokerRequestBatch(this) private val brokerChangeListener = new BrokerChangeListener(this, eventManager) private val topicChangeListener = new TopicChangeListener(this, eventManager) @@ -398,47 +398,52 @@ class KafkaController(val config: KafkaConfig, zkUtils: ZkUtils, time: Time, met } } - /** - * This callback is invoked by the replica state machine's broker change listener with the list of failed brokers - * as input. It does the following - - * 1. Mark partitions with dead leaders as offline - * 2. Triggers the OnlinePartition state change for all new/offline partitions - * 3. Invokes the OfflineReplica state change on the input list of newly started brokers - * 4. If no partitions are effected then send UpdateMetadataRequest to live or shutting down brokers - * - * Note that we don't need to refresh the leader/isr cache for all topic/partitions at this point. This is because - * the partition state machine will refresh our cache for us when performing leader election for all new/offline - * partitions coming online. - */ def onBrokerFailure(deadBrokers: Seq[Int]) { info("Broker failure callback for %s".format(deadBrokers.mkString(","))) deadBrokers.foreach(controllerContext.replicasOnOfflineDisks.remove) val deadBrokersThatWereShuttingDown = deadBrokers.filter(id => controllerContext.shuttingDownBrokerIds.remove(id)) info("Removed %s from list of shutting down brokers.".format(deadBrokersThatWereShuttingDown)) - val deadBrokersSet = deadBrokers.toSet - // trigger OfflinePartition state for all partitions whose current leader is one amongst the dead brokers + val allReplicasOnDeadBrokers = controllerContext.replicasOnBrokers(deadBrokers.toSet) + onReplicaBecomeOffline(allReplicasOnDeadBrokers) + } + + /** + * This callback is invoked by the replica state machine's broker change listener with the list of failed brokers + * as input. It does the following - + * 1. Mark partitions with dead leaders as offline + * 2. Triggers the OnlinePartition state change for all new/offline partitions + * 3. Invokes the OfflineReplica state change on the input list of newly started brokers + * 4. If no partitions are effected then send UpdateMetadataRequest to live or shutting down brokers + * + * Note that we don't need to refresh the leader/isr cache for all topic/partitions at this point. This is because + * the partition state machine will refresh our cache for us when performing leader election for all new/offline + * partitions coming online. + */ + def onReplicaBecomeOffline(newOfflineReplicas: Set[PartitionAndReplica]): Unit = { + val (newOfflineReplicasForDeletion, newOfflineReplicasNotForDeletion) = + newOfflineReplicas.partition(p => topicDeletionManager.isTopicQueuedUpForDeletion(p.topic)) + val partitionsWithoutLeader = controllerContext.partitionLeadershipInfo.filter(partitionAndLeader => - deadBrokersSet.contains(partitionAndLeader._2.leaderAndIsr.leader) && + !controllerContext.isReplicaOnline(partitionAndLeader._2.leaderAndIsr.leader, partitionAndLeader._1) && !topicDeletionManager.isTopicQueuedUpForDeletion(partitionAndLeader._1.topic)).keySet + + // trigger OfflinePartition state for all partitions whose current leader is one amongst the newOfflineReplicas partitionStateMachine.handleStateChanges(partitionsWithoutLeader, OfflinePartition) // trigger OnlinePartition state changes for offline or new partitions partitionStateMachine.triggerOnlinePartitionStateChange() - // filter out the replicas that belong to topics that are being deleted - var allReplicasOnDeadBrokers = controllerContext.replicasOnBrokers(deadBrokersSet) - val activeReplicasOnDeadBrokers = allReplicasOnDeadBrokers.filterNot(p => topicDeletionManager.isTopicQueuedUpForDeletion(p.topic)) - // handle dead replicas - replicaStateMachine.handleStateChanges(activeReplicasOnDeadBrokers, OfflineReplica) - // check if topic deletion state for the dead replicas needs to be updated - val replicasForTopicsToBeDeleted = allReplicasOnDeadBrokers.filter(p => topicDeletionManager.isTopicQueuedUpForDeletion(p.topic)) - if(replicasForTopicsToBeDeleted.nonEmpty) { + // trigger OfflineReplica state change for those newly-discovered offline replicas + replicaStateMachine.handleStateChanges(newOfflineReplicasNotForDeletion, OfflineReplica) + + // fail deletion of topics that affected by the offline replicas + if (newOfflineReplicasForDeletion.nonEmpty) { // it is required to mark the respective replicas in TopicDeletionFailed state since the replica cannot be - // deleted when the broker is down. This will prevent the replica from being in TopicDeletionStarted state indefinitely + // deleted when its disk is down. This will prevent the replica from being in TopicDeletionStarted state indefinitely // since topic deletion cannot be retried until at least one replica is in TopicDeletionStarted state - topicDeletionManager.failReplicaDeletion(replicasForTopicsToBeDeleted) + topicDeletionManager.failReplicaDeletion(newOfflineReplicasForDeletion) } - // If broker failure did not require leader re-election, inform brokers of failed broker + // If replica failure did not require leader re-election, inform brokers of the offline replica // Note that during leader re-election, brokers update their metadata if (partitionsWithoutLeader.isEmpty) { sendUpdateMetadataRequest(controllerContext.liveOrShuttingDownBrokerIds.toSeq) @@ -1522,31 +1527,10 @@ class KafkaController(val config: KafkaConfig, zkUtils: ZkUtils, time: Time, met val currentOfflineReplicas = previousOfflineReplicas -- onlineReplicas ++ offlineReplicas controllerContext.replicasOnOfflineDisks.put(brokerId, currentOfflineReplicas) val newOfflineReplicas = (currentOfflineReplicas -- previousOfflineReplicas).map(tp => PartitionAndReplica(tp.topic, tp.partition, brokerId)) - stateChangeLogger.info(s"Mark replicas ${currentOfflineReplicas -- previousOfflineReplicas} on broker ${brokerId} as offline") - - if (newOfflineReplicas.nonEmpty) { - val (newOfflineReplicasForDeletion, newOfflineReplicasNotForDeletion) = - newOfflineReplicas.partition(p => topicDeletionManager.isTopicQueuedUpForDeletion(p.topic)) - - // trigger OfflinePartition state change for all partitions whose leader is on the failed disk - val partitionsWithoutLeader = controllerContext.partitionLeadershipInfo.filter(partitionAndLeader => - !controllerContext.isReplicaOnline(partitionAndLeader._2.leaderAndIsr.leader, partitionAndLeader._1) && - !topicDeletionManager.isTopicQueuedUpForDeletion(partitionAndLeader._1.topic)).keySet - partitionStateMachine.handleStateChanges(partitionsWithoutLeader, OfflinePartition) - // trigger OnlinePartition state change for offline or new partitions - partitionStateMachine.triggerOnlinePartitionStateChange() - - // trigger OfflineReplica state change for those newly-discovered offline replicas - replicaStateMachine.handleStateChanges(newOfflineReplicasNotForDeletion, OfflineReplica) - - // fail deletion of topics that affected by the offline replicas - if (newOfflineReplicasForDeletion.nonEmpty) { - // it is required to mark the respective replicas in TopicDeletionFailed state since the replica cannot be - // deleted when its disk is down. This will prevent the replica from being in TopicDeletionStarted state indefinitely - // since topic deletion cannot be retried until at least one replica is in TopicDeletionStarted state - topicDeletionManager.failReplicaDeletion(newOfflineReplicasForDeletion) - } - } + stateChangeLogger.info(s"Mark replicas ${currentOfflineReplicas -- previousOfflineReplicas} on broker $brokerId as offline") + + if (newOfflineReplicas.nonEmpty) + onReplicaBecomeOffline(newOfflineReplicas) } } diff --git a/core/src/main/scala/kafka/controller/PartitionStateMachine.scala b/core/src/main/scala/kafka/controller/PartitionStateMachine.scala index 20501fd204c5..d74163274b79 100755 --- a/core/src/main/scala/kafka/controller/PartitionStateMachine.scala +++ b/core/src/main/scala/kafka/controller/PartitionStateMachine.scala @@ -37,12 +37,12 @@ import scala.collection._ * 4. OfflinePartition : If, after successful leader election, the leader for partition dies, then the partition * moves to the OfflinePartition state. Valid previous states are NewPartition/OnlinePartition */ -class PartitionStateMachine(controller: KafkaController, eventManager: ControllerEventManager) extends Logging { +class PartitionStateMachine(controller: KafkaController) extends Logging { private val controllerContext = controller.controllerContext private val controllerId = controller.config.brokerId private val zkUtils = controllerContext.zkUtils private val partitionState: mutable.Map[TopicAndPartition, PartitionState] = mutable.Map.empty - private val brokerRequestBatch = new ControllerBrokerRequestBatch(controller, eventManager) + private val brokerRequestBatch = new ControllerBrokerRequestBatch(controller) private val noOpPartitionLeaderSelector = new NoOpLeaderSelector(controllerContext) private val stateChangeLogger = KafkaController.stateChangeLogger diff --git a/core/src/main/scala/kafka/controller/ReplicaStateMachine.scala b/core/src/main/scala/kafka/controller/ReplicaStateMachine.scala index 3b8f94c8f8fc..cd2df7107a3e 100755 --- a/core/src/main/scala/kafka/controller/ReplicaStateMachine.scala +++ b/core/src/main/scala/kafka/controller/ReplicaStateMachine.scala @@ -40,12 +40,12 @@ import scala.collection._ * 7. NonExistentReplica: If a replica is deleted successfully, it is moved to this state. Valid previous state is * ReplicaDeletionSuccessful */ -class ReplicaStateMachine(controller: KafkaController, eventManager: ControllerEventManager) extends Logging { +class ReplicaStateMachine(controller: KafkaController) extends Logging { private val controllerContext = controller.controllerContext private val controllerId = controller.config.brokerId private val zkUtils = controllerContext.zkUtils private val replicaState: mutable.Map[PartitionAndReplica, ReplicaState] = mutable.Map.empty - private val brokerRequestBatch = new ControllerBrokerRequestBatch(controller, eventManager) + private val brokerRequestBatch = new ControllerBrokerRequestBatch(controller) private val stateChangeLogger = KafkaController.stateChangeLogger diff --git a/core/src/main/scala/kafka/log/LogManager.scala b/core/src/main/scala/kafka/log/LogManager.scala index f55db5ec6c7a..ffd887e8df20 100755 --- a/core/src/main/scala/kafka/log/LogManager.scala +++ b/core/src/main/scala/kafka/log/LogManager.scala @@ -138,9 +138,7 @@ class LogManager(private val logDirs: Array[File], } info(s"Stopping serving logs in dir $dir") - val newOfflineDirs = liveLogDirs.filter(_.getAbsolutePath == dir) - liveLogDirs --= newOfflineDirs - + liveLogDirs -= new File(dir) if (liveLogDirs.isEmpty) { fatal(s"Shutdown broker because all log dirs in ${logDirs.mkString(", ")} have failed"); Runtime.getRuntime().halt(1) @@ -179,6 +177,34 @@ class LogManager(private val logDirs: Array[File], } } + private def loadLogs(logDir: File, recoveryPoints: Map[TopicPartition, Long], logStartOffsets: Map[TopicPartition, Long]): Unit = { + debug("Loading log '" + logDir.getName + "'") + val topicPartition = Log.parseTopicPartitionName(logDir) + val config = topicConfigs.getOrElse(topicPartition.topic, defaultConfig) + val logRecoveryPoint = recoveryPoints.getOrElse(topicPartition, 0L) + val logStartOffset = logStartOffsets.getOrElse(topicPartition, 0L) + + val current = new Log( + dir = logDir, + config = config, + logStartOffset = logStartOffset, + recoveryPoint = logRecoveryPoint, + maxProducerIdExpirationMs = maxPidExpirationMs, + scheduler = scheduler, + time = time, + brokerTopicStats = brokerTopicStats) + if (logDir.getName.endsWith(Log.DeleteDirSuffix)) { + this.logsToBeDeleted.add(current) + } else { + val previous = this.logs.put(topicPartition, current) + if (previous != null) { + throw new IllegalArgumentException( + "Duplicate log directories found: %s, %s!".format( + current.dir.getAbsolutePath, previous.dir.getAbsolutePath)) + } + } + } + /** * Recover and load all logs in the given data directories */ @@ -186,6 +212,7 @@ class LogManager(private val logDirs: Array[File], info("Loading logs.") val startMs = time.milliseconds val threadPools = ArrayBuffer.empty[ExecutorService] + val offlineDirs = ArrayBuffer.empty[String] val jobs = mutable.Map.empty[File, Seq[Future[_]]] for (dir <- liveLogDirs) { @@ -226,49 +253,31 @@ class LogManager(private val logDirs: Array[File], logDir <- dirContent if logDir.isDirectory } yield { CoreUtils.runnable { - debug("Loading log '" + logDir.getName + "'") - - val topicPartition = Log.parseTopicPartitionName(logDir) - val config = topicConfigs.getOrElse(topicPartition.topic, defaultConfig) - val logRecoveryPoint = recoveryPoints.getOrElse(topicPartition, 0L) - val logStartOffset = logStartOffsets.getOrElse(topicPartition, 0L) - - val current = Log( - dir = logDir, - config = config, - logStartOffset = logStartOffset, - recoveryPoint = logRecoveryPoint, - maxProducerIdExpirationMs = maxPidExpirationMs, - scheduler = scheduler, - time = time, - brokerTopicStats = brokerTopicStats) - if (logDir.getName.endsWith(Log.DeleteDirSuffix)) { - this.logsToBeDeleted.add(current) - } else { - val previous = this.logs.put(topicPartition, current) - if (previous != null) { - throw new IllegalArgumentException( - "Duplicate log directories found: %s, %s!".format( - current.dir.getAbsolutePath, previous.dir.getAbsolutePath)) + try { + loadLogs(logDir, recoveryPoints, logStartOffsets) + } catch { + case e: IOException => + offlineDirs.append(dir.getAbsolutePath) + error("Error while load log dir " + dir.getAbsolutePath, e) } } } - } threadPools.append(pool) jobs(cleanShutdownFile) = jobsForDir.map(pool.submit).toSeq } catch { case e: IOException => + offlineDirs.append(dir.getAbsolutePath) error("Error while load log dir " + dir.getAbsolutePath, e) } } - try { for ((cleanShutdownFile, dirJobs) <- jobs) { dirJobs.foreach(_.get) cleanShutdownFile.delete() } + offlineDirs.foreach(handleLogDirFailure) } catch { case e: ExecutionException => { error("There was an error in one of the threads during logs loading: " + e.getCause) @@ -411,6 +420,7 @@ class LogManager(private val logDirs: Array[File], /** * Delete all data in a partition and start the log at the new offset + * * @param newOffset The new offset to start the log with */ def truncateFullyAndStartAt(topicPartition: TopicPartition, newOffset: Long) { @@ -552,6 +562,7 @@ class LogManager(private val logDirs: Array[File], /** * Rename the directory of the given topic-partition "logdir" as "logdir.uuid.delete" and * add it in the queue for deletion. + * * @param topicPartition TopicPartition that needs to be deleted */ def asyncDelete(topicPartition: TopicPartition) = { diff --git a/core/src/main/scala/kafka/server/ReplicaManager.scala b/core/src/main/scala/kafka/server/ReplicaManager.scala index 87f452504496..d5dfbc5268b7 100644 --- a/core/src/main/scala/kafka/server/ReplicaManager.scala +++ b/core/src/main/scala/kafka/server/ReplicaManager.scala @@ -283,6 +283,7 @@ class ReplicaManager(val config: KafkaConfig, def stopReplica(topicPartition: TopicPartition, deletePartition: Boolean): Errors = { stateChangeLogger.trace(s"Broker $localBrokerId handling stop replica (delete=$deletePartition) for partition $topicPartition") + // TODO: return error if replica is on offline disk val error = Errors.NONE getPartition(topicPartition) match { case Some(_) => @@ -300,7 +301,7 @@ class ReplicaManager(val config: KafkaConfig, // This could happen when topic is being deleted while broker is down and recovers. if (deletePartition && logManager.getLog(topicPartition).isDefined) logManager.asyncDelete(topicPartition) - stateChangeLogger.trace(s"Broker $localBrokerId ignoring stop replica (delete=$deletePartition) for partition $topicPartition as replica doesn't exist on broker") + stateChangeLogger.trace(s"Broker $localBrokerId ignoring stop replica (delete=$deletePartition) for partition $topicPartition as replica either doesn't exist on broker") } stateChangeLogger.trace(s"Broker $localBrokerId finished handling stop replica (delete=$deletePartition) for partition $topicPartition") error diff --git a/core/src/test/scala/integration/kafka/api/AuthorizerIntegrationTest.scala b/core/src/test/scala/integration/kafka/api/AuthorizerIntegrationTest.scala index 71c0f42cd052..392460ca5e67 100644 --- a/core/src/test/scala/integration/kafka/api/AuthorizerIntegrationTest.scala +++ b/core/src/test/scala/integration/kafka/api/AuthorizerIntegrationTest.scala @@ -305,7 +305,7 @@ class AuthorizerIntegrationTest extends BaseRequestTest { private def leaveGroupRequest = new LeaveGroupRequest.Builder(group, "").build() private def leaderAndIsrRequest = { - new requests.LeaderAndIsrRequest.Builder(brokerId, Int.MaxValue, + new requests.LeaderAndIsrRequest.Builder(ApiKeys.LEADER_AND_ISR.latestVersion, brokerId, Int.MaxValue, Map(tp -> new PartitionState(Int.MaxValue, brokerId, Int.MaxValue, List(brokerId).asJava, 2, Seq(brokerId).asJava, false)).asJava, Set(new Node(brokerId, "localhost", 0)).asJava).build() } diff --git a/core/src/test/scala/unit/kafka/server/LeaderElectionTest.scala b/core/src/test/scala/unit/kafka/server/LeaderElectionTest.scala index d59f1da391c7..b1fa8006df0b 100755 --- a/core/src/test/scala/unit/kafka/server/LeaderElectionTest.scala +++ b/core/src/test/scala/unit/kafka/server/LeaderElectionTest.scala @@ -147,8 +147,9 @@ class LeaderElectionTest extends ZooKeeperTestHarness { Seq(brokerId1, brokerId2).map(Integer.valueOf).asJava, LeaderAndIsr.initialZKVersion, Seq(0, 1).map(Integer.valueOf).asJava, false) ) + val version = ApiKeys.LEADER_AND_ISR.latestVersion val requestBuilder = new LeaderAndIsrRequest.Builder( - controllerId, staleControllerEpoch, partitionStates.asJava, nodes.toSet.asJava) + version, controllerId, staleControllerEpoch, partitionStates.asJava, nodes.toSet.asJava) controllerChannelManager.sendRequest(brokerId2, ApiKeys.LEADER_AND_ISR, requestBuilder, staleControllerEpochCallback) diff --git a/core/src/test/scala/unit/kafka/server/ReplicaManagerTest.scala b/core/src/test/scala/unit/kafka/server/ReplicaManagerTest.scala index 32024d6c03f4..500ac2ec673d 100644 --- a/core/src/test/scala/unit/kafka/server/ReplicaManagerTest.scala +++ b/core/src/test/scala/unit/kafka/server/ReplicaManagerTest.scala @@ -28,7 +28,7 @@ import kafka.utils.timer.MockTimer import kafka.zk.ZooKeeperTestHarness import org.I0Itec.zkclient.ZkClient import org.apache.kafka.common.metrics.Metrics -import org.apache.kafka.common.protocol.Errors +import org.apache.kafka.common.protocol.{ApiKeys, Errors} import org.apache.kafka.common.record._ import org.apache.kafka.common.requests.{IsolationLevel, LeaderAndIsrRequest, PartitionState} import org.apache.kafka.common.requests.ProduceResponse.PartitionResponse @@ -145,7 +145,7 @@ class ReplicaManagerTest extends ZooKeeperTestHarness { val partition = rm.getOrCreatePartition(new TopicPartition(topic, 0)) partition.getOrCreateReplica(0) // Make this replica the leader. - val leaderAndIsrRequest1 = new LeaderAndIsrRequest.Builder(0, 0, + val leaderAndIsrRequest1 = new LeaderAndIsrRequest.Builder(ApiKeys.LEADER_AND_ISR.latestVersion, 0, 0, collection.immutable.Map(new TopicPartition(topic, 0) -> new PartitionState(0, 0, 0, brokerList, 0, brokerList, false)).asJava, Set(new Node(0, "host1", 0), new Node(1, "host2", 1)).asJava).build() rm.becomeLeaderOrFollower(0, leaderAndIsrRequest1, (_, _) => ()) @@ -162,7 +162,7 @@ class ReplicaManagerTest extends ZooKeeperTestHarness { assertFalse(fetchResult.isFired) // Make this replica the follower - val leaderAndIsrRequest2 = new LeaderAndIsrRequest.Builder(0, 0, + val leaderAndIsrRequest2 = new LeaderAndIsrRequest.Builder(ApiKeys.LEADER_AND_ISR.latestVersion, 0, 0, collection.immutable.Map(new TopicPartition(topic, 0) -> new PartitionState(0, 1, 1, brokerList, 0, brokerList, false)).asJava, Set(new Node(0, "host1", 0), new Node(1, "host2", 1)).asJava).build() rm.becomeLeaderOrFollower(1, leaderAndIsrRequest2, (_, _) => ()) @@ -200,7 +200,7 @@ class ReplicaManagerTest extends ZooKeeperTestHarness { partition.getOrCreateReplica(0) // Make this replica the leader. - val leaderAndIsrRequest1 = new LeaderAndIsrRequest.Builder(0, 0, + val leaderAndIsrRequest1 = new LeaderAndIsrRequest.Builder(ApiKeys.LEADER_AND_ISR.latestVersion, 0, 0, collection.immutable.Map(new TopicPartition(topic, 0) -> new PartitionState(0, 0, 0, brokerList, 0, brokerList, true)).asJava, Set(new Node(0, "host1", 0), new Node(1, "host2", 1)).asJava).build() replicaManager.becomeLeaderOrFollower(0, leaderAndIsrRequest1, (_, _) => ()) @@ -289,7 +289,7 @@ class ReplicaManagerTest extends ZooKeeperTestHarness { partition.getOrCreateReplica(0) // Make this replica the leader. - val leaderAndIsrRequest1 = new LeaderAndIsrRequest.Builder(0, 0, + val leaderAndIsrRequest1 = new LeaderAndIsrRequest.Builder(ApiKeys.LEADER_AND_ISR.latestVersion, 0, 0, collection.immutable.Map(new TopicPartition(topic, 0) -> new PartitionState(0, 0, 0, brokerList, 0, brokerList, true)).asJava, Set(new Node(0, "host1", 0), new Node(1, "host2", 1)).asJava).build() replicaManager.becomeLeaderOrFollower(0, leaderAndIsrRequest1, (_, _) => ()) @@ -368,7 +368,7 @@ class ReplicaManagerTest extends ZooKeeperTestHarness { partition.getOrCreateReplica(0) // Make this replica the leader. - val leaderAndIsrRequest1 = new LeaderAndIsrRequest.Builder(0, 0, + val leaderAndIsrRequest1 = new LeaderAndIsrRequest.Builder(ApiKeys.LEADER_AND_ISR.latestVersion, 0, 0, collection.immutable.Map(new TopicPartition(topic, 0) -> new PartitionState(0, 0, 0, brokerList, 0, brokerList, false)).asJava, Set(new Node(0, "host1", 0), new Node(1, "host2", 1), new Node(2, "host2", 2)).asJava).build() rm.becomeLeaderOrFollower(0, leaderAndIsrRequest1, (_, _) => ()) diff --git a/core/src/test/scala/unit/kafka/server/RequestQuotaTest.scala b/core/src/test/scala/unit/kafka/server/RequestQuotaTest.scala index c961c9b74c8b..6c420e1e7004 100644 --- a/core/src/test/scala/unit/kafka/server/RequestQuotaTest.scala +++ b/core/src/test/scala/unit/kafka/server/RequestQuotaTest.scala @@ -180,7 +180,7 @@ class RequestQuotaTest extends BaseRequestTest { .setTargetTimes(Map(tp -> (0L: java.lang.Long)).asJava) case ApiKeys.LEADER_AND_ISR => - new LeaderAndIsrRequest.Builder(brokerId, Int.MaxValue, + new LeaderAndIsrRequest.Builder(ApiKeys.LEADER_AND_ISR.latestVersion, brokerId, Int.MaxValue, Map(tp -> new PartitionState(Int.MaxValue, brokerId, Int.MaxValue, List(brokerId).asJava, 2, Seq(brokerId).asJava, true)).asJava, Set(new Node(brokerId, "localhost", 0)).asJava) From c013a815d72fa85175dfee8816366a02323526e2 Mon Sep 17 00:00:00 2001 From: Dong Lin Date: Mon, 12 Jun 2017 02:13:28 -0700 Subject: [PATCH 03/11] StopReplicaResponse should specify error if replica-to-be-deleted is not found and there is offline directory --- core/src/main/scala/kafka/cluster/Partition.scala | 3 ++- core/src/main/scala/kafka/log/LogManager.scala | 4 +++- core/src/main/scala/kafka/server/ReplicaManager.scala | 8 +++++--- 3 files changed, 10 insertions(+), 5 deletions(-) diff --git a/core/src/main/scala/kafka/cluster/Partition.scala b/core/src/main/scala/kafka/cluster/Partition.scala index fa12184402c9..0720fce4879d 100755 --- a/core/src/main/scala/kafka/cluster/Partition.scala +++ b/core/src/main/scala/kafka/cluster/Partition.scala @@ -151,8 +151,9 @@ class Partition(val topic: String, assignedReplicaMap.clear() inSyncReplicas = Set.empty[Replica] leaderReplicaIdOpt = None - logManager.asyncDelete(topicPartition) removePartitionMetrics() + // This call may throw exception if the log is on offline directory + logManager.asyncDelete(topicPartition) } } diff --git a/core/src/main/scala/kafka/log/LogManager.scala b/core/src/main/scala/kafka/log/LogManager.scala index ffd887e8df20..f67796015b57 100755 --- a/core/src/main/scala/kafka/log/LogManager.scala +++ b/core/src/main/scala/kafka/log/LogManager.scala @@ -502,7 +502,7 @@ class LogManager(private val logDirs: Array[File], getLog(topicPartition).getOrElse { // create the log if it has not already been created in another thread if (!isNew && offlineLogDirs.nonEmpty) - throw new KafkaStorageException("Can not create log for " + topicPartition + " because log directories " + offlineLogDirs + " are offline") + throw new KafkaStorageException("Can not create log for " + topicPartition + " because log directories " + offlineLogDirs.mkString(",") + " are offline") val dataDir = nextLogDir() val dir = new File(dataDir, topicPartition.topic + "-" + topicPartition.partition) @@ -594,6 +594,8 @@ class LogManager(private val logDirs: Array[File], } else { throw new KafkaStorageException("Failed to rename log directory from " + removedLog.dir.getAbsolutePath + " to " + renamedDir.getAbsolutePath) } + } else if (offlineLogDirs.nonEmpty) { + throw new KafkaStorageException("Can not delete log for " + topicPartition + " because it may be on offline directories " + offlineLogDirs.mkString(",")) } } diff --git a/core/src/main/scala/kafka/server/ReplicaManager.scala b/core/src/main/scala/kafka/server/ReplicaManager.scala index d5dfbc5268b7..7a6f988ef027 100644 --- a/core/src/main/scala/kafka/server/ReplicaManager.scala +++ b/core/src/main/scala/kafka/server/ReplicaManager.scala @@ -283,23 +283,25 @@ class ReplicaManager(val config: KafkaConfig, def stopReplica(topicPartition: TopicPartition, deletePartition: Boolean): Errors = { stateChangeLogger.trace(s"Broker $localBrokerId handling stop replica (delete=$deletePartition) for partition $topicPartition") - // TODO: return error if replica is on offline disk val error = Errors.NONE getPartition(topicPartition) match { case Some(_) => if (deletePartition) { val removedPartition = allPartitions.remove(topicPartition) if (removedPartition != null) { - removedPartition.delete() // this will delete the local log val topicHasPartitions = allPartitions.keys.exists(tp => topicPartition.topic == tp.topic) if (!topicHasPartitions) brokerTopicStats.removeMetrics(topicPartition.topic) + // this will delete the local log. This call may throw exception if the log is on offline directory + removedPartition.delete() + } else { + logManager.asyncDelete(topicPartition) } } case None => // Delete log and corresponding folders in case replica manager doesn't hold them anymore. // This could happen when topic is being deleted while broker is down and recovers. - if (deletePartition && logManager.getLog(topicPartition).isDefined) + if (deletePartition) logManager.asyncDelete(topicPartition) stateChangeLogger.trace(s"Broker $localBrokerId ignoring stop replica (delete=$deletePartition) for partition $topicPartition as replica either doesn't exist on broker") } From 15ab857bb60755a1e5c296dabf59d1d43f44fc0f Mon Sep 17 00:00:00 2001 From: Dong Lin Date: Tue, 13 Jun 2017 13:15:58 -0700 Subject: [PATCH 04/11] Address comments --- core/src/main/scala/kafka/controller/KafkaController.scala | 2 -- 1 file changed, 2 deletions(-) diff --git a/core/src/main/scala/kafka/controller/KafkaController.scala b/core/src/main/scala/kafka/controller/KafkaController.scala index 1e63b6ee9fa3..af8380621f99 100644 --- a/core/src/main/scala/kafka/controller/KafkaController.scala +++ b/core/src/main/scala/kafka/controller/KafkaController.scala @@ -1696,8 +1696,6 @@ class TopicChangeListener(controller: KafkaController, eventManager: ControllerE /** * Called when broker notifies controller of disk change - * - * @param controller */ class LogDirEventNotificationListener(controller: KafkaController, eventManager: ControllerEventManager) extends IZkChildListener with Logging { override def handleChildChange(parentPath: String, currentChilds: java.util.List[String]): Unit = { From 563bf001b78662834e403619657f21415095d38c Mon Sep 17 00:00:00 2001 From: Dong Lin Date: Fri, 16 Jun 2017 19:35:18 -0700 Subject: [PATCH 05/11] Address comments --- .../main/scala/kafka/admin/AdminUtils.scala | 2 +- .../src/main/scala/kafka/log/LogManager.scala | 19 +++++-- .../main/scala/kafka/server/KafkaServer.scala | 35 +++++++------ .../scala/kafka/server/ReplicaManager.scala | 2 + .../scala/unit/kafka/utils/TestUtils.scala | 1 + docs/upgrade.html | 50 ++++++++++++++++++- 6 files changed, 87 insertions(+), 22 deletions(-) diff --git a/core/src/main/scala/kafka/admin/AdminUtils.scala b/core/src/main/scala/kafka/admin/AdminUtils.scala index 923ceb747fb6..7de85e49a906 100644 --- a/core/src/main/scala/kafka/admin/AdminUtils.scala +++ b/core/src/main/scala/kafka/admin/AdminUtils.scala @@ -408,7 +408,7 @@ object AdminUtils extends Logging with AdminUtilities { zkUtils.pathExists(getTopicPath(topic)) def getBrokerMetadatas(zkUtils: ZkUtils, rackAwareMode: RackAwareMode = RackAwareMode.Enforced, - brokerList: Option[Seq[Int]] = None): Seq[BrokerMetadata] = { + brokerList: Option[Seq[Int]] = None): Seq[BrokerMetadata] = { val allBrokers = zkUtils.getAllBrokersInCluster() val brokers = brokerList.map(brokerIds => allBrokers.filter(b => brokerIds.contains(b.id))).getOrElse(allBrokers) val brokersWithRack = brokers.filter(_.rack.nonEmpty) diff --git a/core/src/main/scala/kafka/log/LogManager.scala b/core/src/main/scala/kafka/log/LogManager.scala index f67796015b57..5158070534b1 100755 --- a/core/src/main/scala/kafka/log/LogManager.scala +++ b/core/src/main/scala/kafka/log/LogManager.scala @@ -21,8 +21,10 @@ import java.io._ import java.nio.file.Files import java.util.concurrent._ +import com.yammer.metrics.core.Gauge import kafka.admin.AdminUtils import kafka.common.{KafkaException, KafkaStorageException} +import kafka.metrics.KafkaMetricsGroup import kafka.server.checkpoints.OffsetCheckpointFile import kafka.server.{BrokerState, RecoveringFromUncleanShutdown, _} import kafka.utils._ @@ -45,6 +47,7 @@ import scala.collection.mutable.ArrayBuffer */ @threadsafe class LogManager(private val logDirs: Array[File], + private val initialOfflineDirs: Array[File], val topicConfigs: Map[String, LogConfig], // note that this doesn't get updated after creation val defaultConfig: LogConfig, val cleanerConfig: CleanerConfig, @@ -59,7 +62,7 @@ class LogManager(private val logDirs: Array[File], brokerTopicStats: BrokerTopicStats, val zkUtils: ZkUtils, val brokerId: Int, - time: Time) extends Logging { + time: Time) extends Logging with KafkaMetricsGroup { val RecoveryPointCheckpointFile = "recovery-point-offset-checkpoint" val LogStartOffsetCheckpointFile = "log-start-offset-checkpoint" val LockFile = ".lock" @@ -69,7 +72,7 @@ class LogManager(private val logDirs: Array[File], private val logs = new Pool[TopicPartition, Log]() private val logsToBeDeleted = new LinkedBlockingQueue[Log]() - val liveLogDirs: ArrayBuffer[File] = createAndValidateLogDirs(logDirs) + val liveLogDirs: ArrayBuffer[File] = createAndValidateLogDirs(logDirs, initialOfflineDirs) @volatile private var recoveryPointCheckpoints = liveLogDirs.map(dir => (dir, new OffsetCheckpointFile(new File(dir, RecoveryPointCheckpointFile)))).toMap @volatile private var logStartOffsetCheckpoints = liveLogDirs.map(dir => (dir, new OffsetCheckpointFile(new File(dir, LogStartOffsetCheckpointFile)))).toMap @@ -86,6 +89,12 @@ class LogManager(private val logDirs: Array[File], else null + val offlineLogDirectoryCount = newGauge( + "OfflineLogDirectoryCount", + new Gauge[Int] { + def value = offlineLogDirs.length + } + ) /** * Create and check validity of the given directories, specifically: @@ -95,13 +104,13 @@ class LogManager(private val logDirs: Array[File], *
  • Check that each path is a readable directory * */ - private def createAndValidateLogDirs(dirs: Seq[File]): ArrayBuffer[File] = { + private def createAndValidateLogDirs(dirs: Seq[File], initialOfflineDirs: Seq[File]): ArrayBuffer[File] = { if(dirs.map(_.getCanonicalPath).toSet.size < dirs.size) throw new KafkaException("Duplicate log directory found: " + dirs.mkString(", ")) val liveLogDirs = ArrayBuffer.empty[File] - for (dir <- dirs) { + for (dir <- dirs if !initialOfflineDirs.contains(dir)) { try { if (!dir.exists) { info("Log directory '" + dir.getAbsolutePath + "' not found, creating it.") @@ -677,6 +686,7 @@ class LogManager(private val logDirs: Array[File], object LogManager { def apply(config: KafkaConfig, + initialOfflineDirs: Seq[String], zkUtils: ZkUtils, brokerState: BrokerState, kafkaScheduler: KafkaScheduler, @@ -700,6 +710,7 @@ object LogManager { enableCleaner = config.logCleanerEnable) new LogManager(logDirs = config.logDirs.map(new File(_)).toArray, + initialOfflineDirs = initialOfflineDirs.map(new File(_)).toArray, topicConfigs = topicConfigs, defaultConfig = defaultLogConfig, cleanerConfig = cleanerConfig, diff --git a/core/src/main/scala/kafka/server/KafkaServer.scala b/core/src/main/scala/kafka/server/KafkaServer.scala index dffe6a08a104..d01c5d4ee2c0 100755 --- a/core/src/main/scala/kafka/server/KafkaServer.scala +++ b/core/src/main/scala/kafka/server/KafkaServer.scala @@ -47,7 +47,7 @@ import org.apache.kafka.common.utils.{AppInfoParser, Time} import org.apache.kafka.common.{ClusterResource, Node} import scala.collection.JavaConverters._ -import scala.collection.{Map, mutable} +import scala.collection.{Seq, Map, mutable} object KafkaServer { // Copy the subset of properties that are relevant to Logs @@ -195,7 +195,8 @@ class KafkaServer(val config: KafkaConfig, time: Time = Time.SYSTEM, threadNameP info(s"Cluster ID = $clusterId") /* generate brokerId */ - config.brokerId = getBrokerId + val (brokerId, initialOfflineDirs) = getBrokerIdAndOfflineDirs + config.brokerId = brokerId this.logIdent = "[Kafka Server " + config.brokerId + "], " /* create and configure metrics */ @@ -212,7 +213,7 @@ class KafkaServer(val config: KafkaConfig, time: Time = Time.SYSTEM, threadNameP notifyClusterListeners(kafkaMetricsReporters ++ reporters.asScala) /* start log manager */ - logManager = LogManager(config, zkUtils, brokerState, kafkaScheduler, time, brokerTopicStats) + logManager = LogManager(config, initialOfflineDirs, zkUtils, brokerState, kafkaScheduler, time, brokerTopicStats) logManager.startup() metadataCache = new MetadataCache(config.brokerId) @@ -655,9 +656,10 @@ class KafkaServer(val config: KafkaConfig, time: Time = Time.SYSTEM, threadNameP * * @return A brokerId. */ - private def getBrokerId: Int = { + private def getBrokerIdAndOfflineDirs: (Int, Seq[String]) = { var brokerId = config.brokerId val brokerIdSet = mutable.HashSet[Int]() + val offlineDirs = mutable.ArrayBuffer.empty[String] for (logDir <- config.logDirs) { try { @@ -667,6 +669,7 @@ class KafkaServer(val config: KafkaConfig, time: Time = Time.SYSTEM, threadNameP } } catch { case e : IOException => + offlineDirs += logDir error(s"Fail to read ${brokerMetaPropsFile} under log directory ${logDir}", e) } } @@ -685,20 +688,22 @@ class KafkaServer(val config: KafkaConfig, time: Time = Time.SYSTEM, threadNameP else if(brokerIdSet.size == 1) // pick broker.id from meta.properties brokerId = brokerIdSet.last - brokerId + + (brokerId, offlineDirs) } private def checkpointBrokerId(brokerId: Int) { - for (logDir <- config.logDirs) { - try { - val checkpoint = brokerMetadataCheckpoints(logDir) - val brokerMetadataOpt = checkpoint.read() - if (brokerMetadataOpt.isEmpty) - checkpoint.write(BrokerMetadata(brokerId)) - } catch { - case e : IOException => - error(s"Fail to checkpoint brokerId in ${brokerMetaPropsFile} under log directory ${logDir}", e) - } + var logDirsWithoutMetaProps: List[String] = List() + + for (logDir <- config.logDirs if logManager.liveLogDirs.contains(new File(logDir))) { + val brokerMetadataOpt = brokerMetadataCheckpoints(logDir).read() + if(brokerMetadataOpt.isEmpty) + logDirsWithoutMetaProps ++= List(logDir) + } + + for(logDir <- logDirsWithoutMetaProps) { + val checkpoint = brokerMetadataCheckpoints(logDir) + checkpoint.write(BrokerMetadata(brokerId)) } } diff --git a/core/src/main/scala/kafka/server/ReplicaManager.scala b/core/src/main/scala/kafka/server/ReplicaManager.scala index 7a6f988ef027..f55d4371e1b0 100644 --- a/core/src/main/scala/kafka/server/ReplicaManager.scala +++ b/core/src/main/scala/kafka/server/ReplicaManager.scala @@ -974,6 +974,7 @@ class ReplicaManager(val config: KafkaConfig, stateChangeLogger.error(("Broker %d skipped the become-leader state change with correlation id %d from " + "controller %d epoch %d for partition %s since the replica for the partition is offline due to disk error %s.") .format(localBrokerId, correlationId, controllerId, epoch, partition.topicPartition, e)) + getLogDir(new TopicPartition(partition.topic, partition.partitionId)).foreach(handleLogDirFailure) responseMap.put(new TopicPartition(partition.topic, partition.partitionId), Errors.KAFKA_STORAGE_ERROR) } } @@ -1065,6 +1066,7 @@ class ReplicaManager(val config: KafkaConfig, "controller %d epoch %d for partition [%s,%d] since the replica for the partition is offline due to disk error %s") .format(localBrokerId, correlationId, controllerId, partitionStateInfo.controllerEpoch, partition.topic, partition.partitionId, e)) + getLogDir(new TopicPartition(partition.topic, partition.partitionId)).foreach(handleLogDirFailure) responseMap.put(new TopicPartition(partition.topic, partition.partitionId), Errors.KAFKA_STORAGE_ERROR) } } diff --git a/core/src/test/scala/unit/kafka/utils/TestUtils.scala b/core/src/test/scala/unit/kafka/utils/TestUtils.scala index d9274be0cc5e..e54a1f32410f 100755 --- a/core/src/test/scala/unit/kafka/utils/TestUtils.scala +++ b/core/src/test/scala/unit/kafka/utils/TestUtils.scala @@ -1012,6 +1012,7 @@ object TestUtils extends Logging { time: MockTime = new MockTime(), brokerId: Int = 0): LogManager = { new LogManager(logDirs = logDirs, + initialOfflineDirs = Array.empty[File], topicConfigs = Map(), defaultConfig = defaultConfig, cleanerConfig = cleanerConfig, diff --git a/docs/upgrade.html b/docs/upgrade.html index 98c749c837a5..e47faa033fb6 100644 --- a/docs/upgrade.html +++ b/docs/upgrade.html @@ -17,7 +17,53 @@ -