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

MINOR: A few cleanups for DescribeQuorum APIs #12548

Merged
merged 7 commits into from Aug 24, 2022
Merged
Show file tree
Hide file tree
Changes from 3 commits
Commits
File filter

Filter by extension

Filter by extension

Conversations
Failed to load comments.
Jump to
Jump to file
Failed to load files.
Diff view
Diff view
Expand Up @@ -4357,12 +4357,21 @@ private QuorumInfo.ReplicaState translateReplicaState(DescribeQuorumResponseData
}

private QuorumInfo createQuorumResult(final DescribeQuorumResponseData.PartitionData partition) {
List<QuorumInfo.ReplicaState> voters = partition.currentVoters().stream()
.map(this::translateReplicaState)
.collect(Collectors.toList());

List<QuorumInfo.ReplicaState> observers = partition.observers().stream()
.map(this::translateReplicaState)
.collect(Collectors.toList());

return new QuorumInfo(
partition.leaderId(),
partition.leaderEpoch(),
partition.highWatermark(),
partition.currentVoters().stream().map(v -> translateReplicaState(v)).collect(Collectors.toList()),
partition.observers().stream().map(o -> translateReplicaState(o)).collect(Collectors.toList()));
partition.leaderId(),
partition.leaderEpoch(),
partition.highWatermark(),
voters,
observers
);
}

@Override
Expand Down
Expand Up @@ -24,29 +24,35 @@
* This class is used to describe the state of the quorum received in DescribeQuorumResponse.
*/
public class QuorumInfo {
private final Integer leaderId;
private final Integer leaderEpoch;
private final Long highWatermark;
private final int leaderId;
private final long leaderEpoch;
Copy link
Contributor

Choose a reason for hiding this comment

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

The leaderEpoch is in int32 type, any reason we change to use long here?

{ "name": "LeaderEpoch", "type": "int32", "versions": "0+",
"about": "The latest known leader epoch"},

private final long highWatermark;
private final List<ReplicaState> voters;
private final List<ReplicaState> observers;

QuorumInfo(Integer leaderId, Integer leaderEpoch, Long highWatermark, List<ReplicaState> voters, List<ReplicaState> observers) {
QuorumInfo(
int leaderId,
long leaderEpoch,
long highWatermark,
List<ReplicaState> voters,
List<ReplicaState> observers
) {
this.leaderId = leaderId;
this.leaderEpoch = leaderEpoch;
this.highWatermark = highWatermark;
this.voters = voters;
this.observers = observers;
}

public Integer leaderId() {
public int leaderId() {
return leaderId;
}

public Integer leaderEpoch() {
public long leaderEpoch() {
return leaderEpoch;
}

public Long highWatermark() {
public long highWatermark() {
return highWatermark;
}

Expand All @@ -63,20 +69,24 @@ public boolean equals(Object o) {
if (this == o) return true;
if (o == null || getClass() != o.getClass()) return false;
QuorumInfo that = (QuorumInfo) o;
return leaderId.equals(that.leaderId)
&& voters.equals(that.voters)
&& observers.equals(that.observers);
return leaderId == that.leaderId
&& leaderEpoch == that.leaderEpoch
&& highWatermark == that.highWatermark
&& Objects.equals(voters, that.voters)
&& Objects.equals(observers, that.observers);
}

@Override
public int hashCode() {
return Objects.hash(leaderId, voters, observers);
return Objects.hash(leaderId, leaderEpoch, highWatermark, voters, observers);
}

@Override
public String toString() {
return "QuorumInfo(" +
"leaderId=" + leaderId +
", leaderEpoch=" + leaderEpoch +
", highWatermark=" + highWatermark +
", voters=" + voters +
", observers=" + observers +
')';
Expand All @@ -85,8 +95,8 @@ public String toString() {
public static class ReplicaState {
private final int replicaId;
private final long logEndOffset;
private final OptionalLong lastFetchTimeMs;
private final OptionalLong lastCaughtUpTimeMs;
private final OptionalLong lastFetchTimestamp;
private final OptionalLong lastCaughtUpTimestamp;

ReplicaState() {
this(0, 0, OptionalLong.empty(), OptionalLong.empty());
Expand All @@ -95,13 +105,13 @@ public static class ReplicaState {
ReplicaState(
int replicaId,
long logEndOffset,
OptionalLong lastFetchTimeMs,
OptionalLong lastCaughtUpTimeMs
OptionalLong lastFetchTimestamp,
OptionalLong lastCaughtUpTimestamp
) {
this.replicaId = replicaId;
this.logEndOffset = logEndOffset;
this.lastFetchTimeMs = lastFetchTimeMs;
this.lastCaughtUpTimeMs = lastCaughtUpTimeMs;
this.lastFetchTimestamp = lastFetchTimestamp;
this.lastCaughtUpTimestamp = lastCaughtUpTimestamp;
}

/**
Expand All @@ -125,15 +135,15 @@ public long logEndOffset() {
* @return The value of the lastFetchTime if known, empty otherwise
*/
public OptionalLong lastFetchTimeMs() {
hachikuji marked this conversation as resolved.
Show resolved Hide resolved
return lastFetchTimeMs;
return lastFetchTimestamp;
}

/**
* Return the lastCaughtUpTime in milliseconds for this replica.
* @return The value of the lastCaughtUpTime if known, empty otherwise
*/
public OptionalLong lastCaughtUpTimeMs() {
hachikuji marked this conversation as resolved.
Show resolved Hide resolved
return lastCaughtUpTimeMs;
return lastCaughtUpTimestamp;
}

@Override
Expand All @@ -143,22 +153,22 @@ public boolean equals(Object o) {
ReplicaState that = (ReplicaState) o;
return replicaId == that.replicaId
&& logEndOffset == that.logEndOffset
&& lastFetchTimeMs.equals(that.lastFetchTimeMs)
&& lastCaughtUpTimeMs.equals(that.lastCaughtUpTimeMs);
&& lastFetchTimestamp.equals(that.lastFetchTimestamp)
&& lastCaughtUpTimestamp.equals(that.lastCaughtUpTimestamp);
}

@Override
public int hashCode() {
return Objects.hash(replicaId, logEndOffset, lastFetchTimeMs, lastCaughtUpTimeMs);
return Objects.hash(replicaId, logEndOffset, lastFetchTimestamp, lastCaughtUpTimestamp);
}

@Override
public String toString() {
return "ReplicaState(" +
"replicaId=" + replicaId +
", logEndOffset=" + logEndOffset +
", lastFetchTimeMs=" + lastFetchTimeMs +
", lastCaughtUpTimeMs=" + lastCaughtUpTimeMs +
", lastFetchTimestamp=" + lastFetchTimestamp +
", lastCaughtUpTimestamp=" + lastCaughtUpTimestamp +
')';
}
}
Expand Down
Expand Up @@ -18,15 +18,13 @@

import org.apache.kafka.common.TopicPartition;
import org.apache.kafka.common.message.DescribeQuorumResponseData;
import org.apache.kafka.common.message.DescribeQuorumResponseData.ReplicaState;
import org.apache.kafka.common.protocol.ApiKeys;
import org.apache.kafka.common.protocol.ByteBufferAccessor;
import org.apache.kafka.common.protocol.Errors;

import java.nio.ByteBuffer;
import java.util.Collections;
import java.util.HashMap;
import java.util.List;
import java.util.Map;

/**
Expand Down Expand Up @@ -85,23 +83,15 @@ public static DescribeQuorumResponseData singletonErrorResponse(
}


public static DescribeQuorumResponseData singletonResponse(TopicPartition topicPartition,
int leaderId,
int leaderEpoch,
long highWatermark,
List<ReplicaState> voterStates,
List<ReplicaState> observerStates) {
public static DescribeQuorumResponseData singletonResponse(
TopicPartition topicPartition,
DescribeQuorumResponseData.PartitionData partitionData
) {
return new DescribeQuorumResponseData()
.setTopics(Collections.singletonList(new DescribeQuorumResponseData.TopicData()
.setTopicName(topicPartition.topic())
.setPartitions(Collections.singletonList(new DescribeQuorumResponseData.PartitionData()
.setPartitionIndex(topicPartition.partition())
.setErrorCode(Errors.NONE.code())
.setLeaderId(leaderId)
.setLeaderEpoch(leaderEpoch)
.setHighWatermark(highWatermark)
.setCurrentVoters(voterStates)
.setObservers(observerStates)))));
.setPartitions(Collections.singletonList(partitionData
.setPartitionIndex(topicPartition.partition())))));
}

public static DescribeQuorumResponse parse(ByteBuffer buffer, short version) {
Expand Down
Expand Up @@ -607,7 +607,7 @@ private static ApiVersionsResponse prepareApiVersionsResponseForDescribeFeatures
.setErrorCode(error.code()));
}

private static QuorumInfo defaultQuorumInfo(Boolean emptyOptionals) {
private static QuorumInfo defaultQuorumInfo(boolean emptyOptionals) {
return new QuorumInfo(1, 1, 1L,
singletonList(new QuorumInfo.ReplicaState(1, 100,
emptyOptionals ? OptionalLong.empty() : OptionalLong.of(1000),
Expand Down Expand Up @@ -637,8 +637,8 @@ private static DescribeQuorumResponse prepareDescribeQuorumResponse(
replica.setLastCaughtUpTimestamp(emptyOptionals ? -1 : 1000);
partitions.add(new DescribeQuorumResponseData.PartitionData().setPartitionIndex(partitionIndex)
.setLeaderId(1)
.setLeaderEpoch(0)
.setHighWatermark(0)
.setLeaderEpoch(1)
.setHighWatermark(1)
.setCurrentVoters(singletonList(replica))
.setObservers(singletonList(replica))
.setErrorCode(partitionLevelError.code()));
Expand Down
Expand Up @@ -133,7 +133,7 @@ object MetadataQuorumCommand {
).map(_.toString)
}
prettyPrintTable(
Array("NodeId", "LogEndOffset", "Lag", "LastFetchTimeMs", "LastCaughtUpTimeMs", "Status"),
Array("NodeId", "LogEndOffset", "Lag", "LastFetchTimestamp", "LastCaughtUpTimestamp", "Status"),
(convertQuorumInfo(Seq(leader), "Leader")
++ convertQuorumInfo(quorumInfo.voters.asScala.filter(_.replicaId != leaderId).toSeq, "Follower")
++ convertQuorumInfo(quorumInfo.observers.asScala.toSeq, "Observer")).asJava,
Expand Down
13 changes: 5 additions & 8 deletions raft/src/main/java/org/apache/kafka/raft/KafkaRaftClient.java
Expand Up @@ -275,7 +275,7 @@ private void updateLeaderEndOffsetAndTimestamp(
) {
final LogOffsetMetadata endOffsetMetadata = log.endOffset();

if (state.updateLocalState(currentTimeMs, endOffsetMetadata)) {
if (state.updateLocalState(endOffsetMetadata)) {
onUpdateLeaderHighWatermark(state, currentTimeMs);
}

Expand Down Expand Up @@ -1014,7 +1014,7 @@ private FetchResponseData tryCompleteFetchRequest(
if (validOffsetAndEpoch.kind() == ValidOffsetAndEpoch.Kind.VALID) {
LogFetchInfo info = log.read(fetchOffset, Isolation.UNCOMMITTED);

if (state.updateReplicaState(replicaId, currentTimeMs, info.startOffsetMetadata, log.endOffset().offset)) {
if (state.updateReplicaState(replicaId, currentTimeMs, info.startOffsetMetadata)) {
onUpdateLeaderHighWatermark(state, currentTimeMs);
}

Expand Down Expand Up @@ -1176,12 +1176,9 @@ private DescribeQuorumResponseData handleDescribeQuorumRequest(
}

LeaderState<T> leaderState = quorum.leaderStateOrThrow();
return DescribeQuorumResponse.singletonResponse(log.topicPartition(),
leaderState.localId(),
leaderState.epoch(),
leaderState.highWatermark().isPresent() ? leaderState.highWatermark().get().offset : -1,
leaderState.quorumResponseVoterStates(currentTimeMs),
leaderState.quorumResponseObserverStates(currentTimeMs)
return DescribeQuorumResponse.singletonResponse(
log.topicPartition(),
leaderState.describeQuorum(currentTimeMs)
);
}

Expand Down