Skip to content
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-8442:Inconsistent ISR output in topic command #6836

Merged
merged 4 commits into from Jul 30, 2019

Conversation

huxihx
Copy link
Contributor

@huxihx huxihx commented May 29, 2019

https://issues.apache.org/jira/browse/KAFKA-8442

Modify leader and ISR output for --zookeeper path in TopicCommand, to be consistent with --bootstrap-server path.

More detailed description of your change,
if necessary. The PR title and PR message become
the squashed commit message, so use a separate
comment to ping reviewers.

Summary of testing strategy (including rationale)
for the feature or bug fix. Unit and/or integration
tests are expected for any behaviour change and
system tests should be considered for larger changes.

Committer Checklist (excluded from commit message)

  • Verify design and implementation
  • Verify test coverage and CI build status
  • Verify documentation (including upgrade notes)

@huxihx
Copy link
Contributor Author

huxihx commented May 29, 2019

retest this please

assignedReplicas,
isr = if (leaderIsrEpoch.isEmpty) Seq.empty[Int] else leaderIsrEpoch.get.leaderAndIsr.isr,
isr = if (leaderIsrEpoch.isEmpty) Seq.empty[Int] else leaderIsrEpoch.get.leaderAndIsr.isr.filter(liveBrokers.contains),
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

This is sort of the opposite of what I suggested in the JIRA. It is often still useful to know the ISR information even when the broker is offline. Maybe it would be better for now to list the isr as "not available" when using --bootstrap-server?

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Thanks for the comments. Does it make sense to display a nonempty isr for a partition with no leader actually?
Topic: foo Partition: 0 Leader: -1 Replicas: 1,3 Isr: 1

You said it showed the current ISR correctly. I am very intrigued to know the reason. The output for --bootstrap-server, in my view, reflects the correct state for a partition.

Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

The ISR is still valid if the leader is offline. It tells you which replica must be brought back online in order to avoid the need for unclean leader election.

LeaderAndIsr(leader = 0, leaderEpoch = 0, isr = List(0, 1), zkVersion = 0),
controllerEpoch = 0)
)
topicService.zkClient.createTopicPartitionStatesRaw(leaderIsrAndControllerEpochs, ZkVersion.MatchAnyVersion)
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Rather than manually updating the leader and isr state, which may have unexpected behavior, wouldn't it be easier to set the replica assignment in the create topic command?

@huxihx
Copy link
Contributor Author

huxihx commented May 30, 2019

retest this please

@@ -443,7 +443,7 @@ object TopicCommand extends Logging {
print("\tPartition: " + tp.partition)
print("\tLeader: " + (if(tp.leader.isDefined) tp.leader.get else "none"))
print("\tReplicas: " + tp.assignedReplicas.mkString(","))
print("\tIsr: " + tp.isr.mkString(","))
print("\tIsr: " + (if(tp.isr.isEmpty) "not available" else tp.isr.mkString(",")))
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

The case we're trying to detect is here: https://github.com/apache/kafka/blob/trunk/core/src/main/scala/kafka/server/MetadataCache.scala#L93. Basically when there is no current leader, the broker will return an empty ISR. I think the logic here should match that. So if there is no leader defined, then we can represent the ISR as "not available" or maybe just "N/A". Otherwise, we can print the ISR as it is. Does that make sense?

@huxihx
Copy link
Contributor Author

huxihx commented Jun 4, 2019

retest this please

@@ -443,7 +443,7 @@ object TopicCommand extends Logging {
print("\tPartition: " + tp.partition)
print("\tLeader: " + (if(tp.leader.isDefined) tp.leader.get else "none"))
print("\tReplicas: " + tp.assignedReplicas.mkString(","))
print("\tIsr: " + tp.isr.mkString(","))
print("\tIsr: " + (if(tp.leader.isDefined) tp.isr.mkString(",") else "none"))
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

We still have two problems:

  1. This affects both the output using --zookeeper and --bootstrap-server. As I said in the first comment, it is still useful to display the ISR when the leader is offline if we have it available. The problem here is that we do not have an accurate ISR when --bootstrap-server is used.
  2. The term "none" may cause confusion. We are trying to convey the fact that we just don't know what the current ISR is.

Maybe we should try to approach this from a different perspective. The root of the problem is that the Metadata API does not return the ISR faithfully when there is no leader. Can we change that? The rest of the information is returned accurately including the replica set and the offline replicas. I cannot think of a good reason for the inconsistency. I'd suggest we change the behavior here: https://github.com/apache/kafka/blob/trunk/core/src/main/scala/kafka/server/MetadataCache.scala#L98. Then we do not need to handle the case specially here.

@huxihx
Copy link
Contributor Author

huxihx commented Jul 1, 2019

retest this please

Copy link
Contributor

@hachikuji hachikuji left a comment

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Thanks, left a couple more comments.

@@ -120,7 +120,10 @@ object TopicCommand extends Logging {
opts.reportUnavailablePartitions && hasUnavailablePartitions(partitionDescription)
}
private def hasUnderMinIsrPartitions(partitionDescription: PartitionDescription) = {
partitionDescription.isr.size < partitionDescription.minIsrCount
if (partitionDescription.leader.isDefined)
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Hmm.. I'm not sure I understand the reason for this change. Can you explain?

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

TopicCommandWithAdminClientTest.testDescribeUnderMinIsrPartitionsMixed will fail. If leader is not available, the partition must be a under-ISR one.

Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Hmm, good point. Since min.insync.replicas cannot be 0, I guess we could simplify this.

partitionDescription.leader.isEmpty || partitionDescription.isr.size < partitionDescription.minIsrCount

@@ -91,13 +93,10 @@ class MetadataCache(brokerId: Int) extends Logging {
if (errorUnavailableListeners) Errors.LISTENER_NOT_FOUND else Errors.LEADER_NOT_AVAILABLE
}
new MetadataResponse.PartitionMetadata(error, partitionId.toInt, Node.noNode(),
Optional.empty(), replicaInfo.asJava, java.util.Collections.emptyList(),
Optional.empty(), replicaInfo.asJava, isrInfo.asJava,
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Can we have a test case for this change?

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Do you think MetadataCacheTest#getTopicMetadataPartitionLeaderNotAvailable cover this?

Copy link
Contributor

@hachikuji hachikuji left a comment

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

LGTM

@hachikuji
Copy link
Contributor

retest this please

@hachikuji hachikuji merged commit c6286b2 into apache:trunk Jul 30, 2019
Sign up for free to join this conversation on GitHub. Already have an account? Sign in to comment
Labels
None yet
Projects
None yet
2 participants