-
Notifications
You must be signed in to change notification settings - Fork 14.3k
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
KAFKA-18545 Remove Zookeeper logic from LogManager #18592
Changes from all commits
a9a9d2f
cfc3a3d
5fe1cce
523e5ce
db28b02
22551a4
b15ee2a
135bb03
bb3659a
74500da
File filter
Filter by extension
Conversations
Jump to
Diff view
Diff view
There are no files selected for viewing
Original file line number | Diff line number | Diff line change |
---|---|---|
|
@@ -4188,53 +4188,6 @@ class ReplicaManagerTest { | |
} | ||
} | ||
|
||
@Test | ||
def testPartitionMetadataFileCreatedWithExistingLog(): Unit = { | ||
val replicaManager = setupReplicaManagerWithMockedPurgatories(new MockTimer(time)) | ||
try { | ||
val brokerList = Seq[Integer](0, 1).asJava | ||
val topicPartition = new TopicPartition(topic, 0) | ||
|
||
replicaManager.logManager.getOrCreateLog(topicPartition, isNew = true, topicId = None) | ||
|
||
assertTrue(replicaManager.getLog(topicPartition).isDefined) | ||
var log = replicaManager.getLog(topicPartition).get | ||
assertEquals(None, log.topicId) | ||
assertFalse(log.partitionMetadataFile.get.exists()) | ||
|
||
val topicIds = Collections.singletonMap(topic, Uuid.randomUuid()) | ||
val topicNames = topicIds.asScala.map(_.swap).asJava | ||
|
||
def leaderAndIsrRequest(epoch: Int): LeaderAndIsrRequest = new LeaderAndIsrRequest.Builder(0, 0, brokerEpoch, | ||
Seq(new LeaderAndIsrRequest.PartitionState() | ||
.setTopicName(topic) | ||
.setPartitionIndex(0) | ||
.setControllerEpoch(0) | ||
.setLeader(0) | ||
.setLeaderEpoch(epoch) | ||
.setIsr(brokerList) | ||
.setPartitionEpoch(0) | ||
.setReplicas(brokerList) | ||
.setIsNew(true)).asJava, | ||
topicIds, | ||
Set(new Node(0, "host1", 0), new Node(1, "host2", 1)).asJava).build() | ||
|
||
val response = replicaManager.becomeLeaderOrFollower(0, leaderAndIsrRequest(0), (_, _) => ()) | ||
assertEquals(Errors.NONE, response.partitionErrors(topicNames).get(topicPartition)) | ||
assertFalse(replicaManager.localLog(topicPartition).isEmpty) | ||
val id = topicIds.get(topicPartition.topic()) | ||
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. Looking at this again, does this test actually make sense to keep? It's supposed to test the metadata file get created but we've now removed these assertions. 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. Yeah, I also wasn't sure if removing the assertions make sense here. I was planning to take a closer look. If the assertions are not useful, then we should check if the test is useful. 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.
If we're removing the created file assertion, we should also remove this test. This make sense to me. |
||
log = replicaManager.localLog(topicPartition).get | ||
assertTrue(log.partitionMetadataFile.get.exists()) | ||
val partitionMetadata = log.partitionMetadataFile.get.read() | ||
|
||
// Current version of PartitionMetadataFile is 0. | ||
assertEquals(0, partitionMetadata.version) | ||
assertEquals(id, partitionMetadata.topicId) | ||
} finally { | ||
replicaManager.shutdown(checkpointHW = false) | ||
} | ||
} | ||
|
||
@Test | ||
def testInconsistentIdReturnsError(): Unit = { | ||
val replicaManager = setupReplicaManagerWithMockedPurgatories(new MockTimer(time)) | ||
|
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.
Should we change the topid id so it's not optional?
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.
Although this method doesn't utilize Optional's functionality internally, the calling method
Partition#createLogIfNotExists
uses Optional features, and Optional is still required when passing to theUnifiedLog
constructor later. Therefore, I don't see a compelling reason to avoid using Optional at this point.kafka/core/src/main/scala/kafka/cluster/Partition.scala
Line 463 in 78e3545
kafka/core/src/main/scala/kafka/log/UnifiedLog.scala
Line 191 in 78e3545
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.
Makes sense. Unrelated to this PR, but a good clean-up would be to extract the topic id creation so it's always set by the time the UnifiedLog constructor is called (the way it works now unnecessarily exposes an optional topic id even though it is never really optional with kraft).
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.
It a good idea, I will file a Jira to trace this issue.
Update: https://issues.apache.org/jira/browse/KAFKA-18586
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.
we should do this cleanup after all zk-related paths are removed from
ReplicaManager