-
Notifications
You must be signed in to change notification settings - Fork 15.2k
KAFKA-2960: Clear purgatory for partitions before becoming follower #1018
New issue
Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.
By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.
Already on GitHub? Sign in to your account
Closed
Closed
Changes from all commits
Commits
Show all changes
7 commits
Select commit
Hold shift + click to select a range
6ee590b
KAFKA-2960: Clear purgatory for partition before becoming follower
becketqin 846cf06
Addressed Ismael's comments.
becketqin c2d9e29
Addressed Ismael's comments.
becketqin 9dd1369
Addressed Joel's comments.
becketqin 7589c17
Addressed Ismael's comments.
becketqin 29c8d93
Addressed Joel and Ismael's comments.
becketqin 0642e5f
Addressed Joel and Ismaels' comments.
becketqin File filter
Filter by extension
Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
There are no files selected for viewing
This file contains hidden or bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
This file contains hidden or bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
| Original file line number | Diff line number | Diff line change |
|---|---|---|
|
|
@@ -18,25 +18,28 @@ | |
| package kafka.server | ||
|
|
||
|
|
||
| import kafka.api.SerializationTestUtils | ||
| import kafka.message.{Message, ByteBufferMessageSet} | ||
| import kafka.utils.{ZkUtils, MockScheduler, MockTime, TestUtils} | ||
| import org.apache.kafka.common.requests.ProduceRequest | ||
|
|
||
| import java.util.concurrent.atomic.AtomicBoolean | ||
| import java.io.File | ||
| import java.util.concurrent.atomic.AtomicBoolean | ||
|
|
||
| import kafka.api.{FetchResponsePartitionData, PartitionFetchInfo} | ||
| import kafka.cluster.Broker | ||
| import kafka.common.TopicAndPartition | ||
| import kafka.message.{ByteBufferMessageSet, Message} | ||
| import kafka.utils.{MockScheduler, MockTime, TestUtils, ZkUtils} | ||
| import org.I0Itec.zkclient.ZkClient | ||
| import org.apache.kafka.common.metrics.Metrics | ||
| import org.apache.kafka.common.TopicPartition | ||
| import org.apache.kafka.common.protocol.Errors | ||
| import org.apache.kafka.common.requests.LeaderAndIsrRequest | ||
| import org.apache.kafka.common.requests.LeaderAndIsrRequest.PartitionState | ||
| import org.apache.kafka.common.requests.ProduceResponse.PartitionResponse | ||
| import org.apache.kafka.common.utils.{MockTime => JMockTime} | ||
| import org.apache.kafka.common.{BrokerEndPoint, TopicPartition} | ||
| import org.easymock.EasyMock | ||
| import org.I0Itec.zkclient.ZkClient | ||
| import org.junit.Assert.{assertEquals, assertTrue} | ||
| import org.junit.Test | ||
|
Member
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more.
|
||
|
|
||
| import scala.collection.Map | ||
| import scala.collection.JavaConverters._ | ||
| import scala.collection.Map | ||
|
|
||
| class ReplicaManagerTest { | ||
|
|
||
|
|
@@ -47,9 +50,9 @@ class ReplicaManagerTest { | |
| val props = TestUtils.createBrokerConfig(1, TestUtils.MockZkConnect) | ||
| val config = KafkaConfig.fromProps(props) | ||
| val zkClient = EasyMock.createMock(classOf[ZkClient]) | ||
| val zkUtils = ZkUtils(zkClient, false) | ||
| val zkUtils = ZkUtils(zkClient, isZkSecurityEnabled = false) | ||
| val mockLogMgr = TestUtils.createLogManager(config.logDirs.map(new File(_)).toArray) | ||
| val time: MockTime = new MockTime() | ||
| val time = new MockTime() | ||
| val jTime = new JMockTime | ||
| val metrics = new Metrics | ||
| val rm = new ReplicaManager(config, metrics, time, jTime, zkUtils, new MockScheduler(time), mockLogMgr, | ||
|
|
@@ -73,7 +76,7 @@ class ReplicaManagerTest { | |
| val zkClient = EasyMock.createMock(classOf[ZkClient]) | ||
| val zkUtils = ZkUtils(zkClient, false) | ||
| val mockLogMgr = TestUtils.createLogManager(config.logDirs.map(new File(_)).toArray) | ||
| val time: MockTime = new MockTime() | ||
| val time = new MockTime() | ||
| val jTime = new JMockTime | ||
| val metrics = new Metrics | ||
| val rm = new ReplicaManager(config, metrics, time, jTime, zkUtils, new MockScheduler(time), mockLogMgr, | ||
|
|
@@ -84,7 +87,7 @@ class ReplicaManagerTest { | |
| rm.checkpointHighWatermarks() | ||
| } finally { | ||
| // shutdown the replica manager upon test completion | ||
| rm.shutdown(false) | ||
| rm.shutdown(checkpointHW = false) | ||
| metrics.close() | ||
| } | ||
| } | ||
|
|
@@ -94,9 +97,9 @@ class ReplicaManagerTest { | |
| val props = TestUtils.createBrokerConfig(1, TestUtils.MockZkConnect) | ||
| val config = KafkaConfig.fromProps(props) | ||
| val zkClient = EasyMock.createMock(classOf[ZkClient]) | ||
| val zkUtils = ZkUtils(zkClient, false) | ||
| val zkUtils = ZkUtils(zkClient, isZkSecurityEnabled = false) | ||
| val mockLogMgr = TestUtils.createLogManager(config.logDirs.map(new File(_)).toArray) | ||
| val time: MockTime = new MockTime() | ||
| val time = new MockTime() | ||
| val jTime = new JMockTime | ||
| val metrics = new Metrics | ||
| val rm = new ReplicaManager(config, metrics, time, jTime, zkUtils, new MockScheduler(time), mockLogMgr, | ||
|
|
@@ -112,10 +115,84 @@ class ReplicaManagerTest { | |
| messagesPerPartition = Map(new TopicPartition("test1", 0) -> new ByteBufferMessageSet(new Message("first message".getBytes))), | ||
| responseCallback = callback) | ||
| } finally { | ||
| rm.shutdown(false) | ||
| rm.shutdown(checkpointHW = false) | ||
| metrics.close() | ||
| } | ||
|
|
||
| TestUtils.verifyNonDaemonThreadsStatus(this.getClass.getName) | ||
| } | ||
|
|
||
| @Test | ||
| def testClearPurgatoryOnBecomingFollower() { | ||
| val props = TestUtils.createBrokerConfig(0, TestUtils.MockZkConnect) | ||
| props.put("log.dir", TestUtils.tempRelativeDir("data").getAbsolutePath) | ||
| val config = KafkaConfig.fromProps(props) | ||
| val zkClient = EasyMock.createMock(classOf[ZkClient]) | ||
| val zkUtils = ZkUtils(zkClient, isZkSecurityEnabled = false) | ||
| val mockLogMgr = TestUtils.createLogManager(config.logDirs.map(new File(_)).toArray) | ||
| val time = new MockTime() | ||
| val jTime = new JMockTime | ||
| val metrics = new Metrics | ||
| val rm = new ReplicaManager(config, metrics, time, jTime, zkUtils, new MockScheduler(time), mockLogMgr, | ||
| new AtomicBoolean(false)) | ||
|
|
||
| try { | ||
| var produceCallbackFired = false | ||
| def produceCallback(responseStatus: Map[TopicPartition, PartitionResponse]) = { | ||
| assertEquals("Should give NotLeaderForPartitionException", Errors.NOT_LEADER_FOR_PARTITION.code, responseStatus.values.head.errorCode) | ||
| produceCallbackFired = true | ||
| } | ||
|
|
||
| var fetchCallbackFired = false | ||
| def fetchCallback(responseStatus: Map[TopicAndPartition, FetchResponsePartitionData]) = { | ||
| assertEquals("Should give NotLeaderForPartitionException", Errors.NOT_LEADER_FOR_PARTITION.code, responseStatus.values.head.error) | ||
| fetchCallbackFired = true | ||
| } | ||
|
|
||
| val aliveBrokers = Seq(new Broker(0, "host0", 0), new Broker(1, "host1", 1)) | ||
| val metadataCache = EasyMock.createMock(classOf[MetadataCache]) | ||
| EasyMock.expect(metadataCache.getAliveBrokers).andReturn(aliveBrokers).anyTimes() | ||
| EasyMock.replay(metadataCache) | ||
|
|
||
| val brokerList : java.util.List[Integer] = Seq[Integer](0, 1).asJava | ||
| val brokerSet : java.util.Set[Integer] = Set[Integer](0, 1).asJava | ||
|
|
||
| val partition = rm.getOrCreatePartition(topic, 0) | ||
| partition.getOrCreateReplica(0) | ||
| // Make this replica the leader. | ||
| val leaderAndIsrRequest1 = new LeaderAndIsrRequest(0, 0, | ||
| collection.immutable.Map(new TopicPartition(topic, 0) -> new PartitionState(0, 0, 0, brokerList, 0, brokerSet)).asJava, | ||
| Set(new BrokerEndPoint(0, "host1", 0), new BrokerEndPoint(1, "host2", 1)).asJava) | ||
| rm.becomeLeaderOrFollower(0, leaderAndIsrRequest1, metadataCache, (_, _) => {}) | ||
| rm.getLeaderReplicaIfLocal(topic, 0) | ||
|
|
||
| // Append a message. | ||
| rm.appendMessages( | ||
| timeout = 1000, | ||
| requiredAcks = -1, | ||
| internalTopicsAllowed = false, | ||
| messagesPerPartition = Map(new TopicPartition(topic, 0) -> new ByteBufferMessageSet(new Message("first message".getBytes))), | ||
| responseCallback = produceCallback) | ||
|
|
||
| // Fetch some messages | ||
| rm.fetchMessages( | ||
| timeout = 1000, | ||
| replicaId = -1, | ||
| fetchMinBytes = 100000, | ||
| fetchInfo = collection.immutable.Map(new TopicAndPartition(topic, 0) -> new PartitionFetchInfo(0, 100000)), | ||
| responseCallback = fetchCallback) | ||
|
|
||
| // Make this replica the follower | ||
| val leaderAndIsrRequest2 = new LeaderAndIsrRequest(0, 0, | ||
| collection.immutable.Map(new TopicPartition(topic, 0) -> new PartitionState(0, 1, 1, brokerList, 0, brokerSet)).asJava, | ||
| Set(new BrokerEndPoint(0, "host1", 0), new BrokerEndPoint(1, "host2", 1)).asJava) | ||
| rm.becomeLeaderOrFollower(1, leaderAndIsrRequest2, metadataCache, (_, _) => {}) | ||
|
|
||
| assertTrue(produceCallbackFired) | ||
| assertTrue(fetchCallbackFired) | ||
| } finally { | ||
| rm.shutdown(checkpointHW = false) | ||
| metrics.close() | ||
| } | ||
| } | ||
| } | ||
Add this suggestion to a batch that can be applied as a single commit.
This suggestion is invalid because no changes were made to the code.
Suggestions cannot be applied while the pull request is closed.
Suggestions cannot be applied while viewing a subset of changes.
Only one suggestion per line can be applied in a batch.
Add this suggestion to a batch that can be applied as a single commit.
Applying suggestions on deleted lines is not supported.
You must change the existing code in this line in order to create a valid suggestion.
Outdated suggestions cannot be applied.
This suggestion has been applied or marked resolved.
Suggestions cannot be applied from pending reviews.
Suggestions cannot be applied on multi-line comments.
Suggestions cannot be applied while the pull request is queued to merge.
Suggestion cannot be applied right now. Please check back later.
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
In DelayedProduce, we don't recheck if the leader has changed or now. Is tryCompleteDelayedProduce() enough to force the produce request to be completed? We probably need to add a forceComplete() in delayedProducePurgatory?
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
I was wondering about this as well, but I think we do this right? i.e., it makes a call to
Partition.checkEnoughReplicasReachOffsetwhich returns(false, Errors.NOT_LEADER_FOR_PARTITION.code)if the leader is non-localThere was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Yes, that's right. Then the patch LGTM.