-
Notifications
You must be signed in to change notification settings - Fork 15k
KAFKA-13958: Expose logdirs total and usable space via Kafka API (KIP… #12248
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
Changes from all commits
f292c1d
a73b96a
ec61022
9af1566
df8aea2
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 |
|---|---|---|
|
|
@@ -20,19 +20,29 @@ | |
| import org.apache.kafka.common.errors.ApiException; | ||
|
|
||
| import java.util.Map; | ||
| import java.util.OptionalLong; | ||
|
|
||
| import static java.util.Collections.unmodifiableMap; | ||
| import static org.apache.kafka.common.requests.DescribeLogDirsResponse.UNKNOWN_VOLUME_BYTES; | ||
|
|
||
| /** | ||
| * A description of a log directory on a particular broker. | ||
| */ | ||
| public class LogDirDescription { | ||
| private final Map<TopicPartition, ReplicaInfo> replicaInfos; | ||
| private final ApiException error; | ||
| private final OptionalLong totalBytes; | ||
| private final OptionalLong usableBytes; | ||
|
|
||
| public LogDirDescription(ApiException error, Map<TopicPartition, ReplicaInfo> replicaInfos) { | ||
| this(error, replicaInfos, UNKNOWN_VOLUME_BYTES, UNKNOWN_VOLUME_BYTES); | ||
| } | ||
|
|
||
| public LogDirDescription(ApiException error, Map<TopicPartition, ReplicaInfo> replicaInfos, long totalBytes, long usableBytes) { | ||
| this.error = error; | ||
| this.replicaInfos = replicaInfos; | ||
| this.totalBytes = (totalBytes == UNKNOWN_VOLUME_BYTES) ? OptionalLong.empty() : OptionalLong.of(totalBytes); | ||
| this.usableBytes = (usableBytes == UNKNOWN_VOLUME_BYTES) ? OptionalLong.empty() : OptionalLong.of(usableBytes); | ||
| } | ||
|
|
||
| /** | ||
|
|
@@ -54,11 +64,29 @@ public Map<TopicPartition, ReplicaInfo> replicaInfos() { | |
| return unmodifiableMap(replicaInfos); | ||
| } | ||
|
|
||
| /** | ||
| * The total size of the volume this log directory is on or empty if the broker did not return a value. | ||
| * For volumes larger than Long.MAX_VALUE, Long.MAX_VALUE is returned. | ||
| */ | ||
| public OptionalLong totalBytes() { | ||
tombentley marked this conversation as resolved.
Show resolved
Hide resolved
|
||
| return totalBytes; | ||
| } | ||
|
|
||
| /** | ||
| * The usable size on the volume this log directory is on or empty if the broker did not return a value. | ||
| * For usable sizes larger than Long.MAX_VALUE, Long.MAX_VALUE is returned. | ||
| */ | ||
| public OptionalLong usableBytes() { | ||
|
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. Javadoc
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. Also do we want to say something about the contraints on usableBytes and totalBytes (individually and compared with each other).
Member
Author
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. I've added some details, let me know if it's not what you had in mind |
||
| return usableBytes; | ||
| } | ||
|
|
||
| @Override | ||
| public String toString() { | ||
| return "LogDirDescription(" + | ||
| "replicaInfos=" + replicaInfos + | ||
| ", error=" + error + | ||
| ", totalBytes=" + totalBytes + | ||
| ", usableBytes=" + usableBytes + | ||
| ')'; | ||
| } | ||
| } | ||
| Original file line number | Diff line number | Diff line change | ||||||||||||||
|---|---|---|---|---|---|---|---|---|---|---|---|---|---|---|---|---|
|
|
@@ -31,6 +31,7 @@ | |||||||||||||||
| public class DescribeLogDirsResponse extends AbstractResponse { | ||||||||||||||||
|
|
||||||||||||||||
| public static final long INVALID_OFFSET_LAG = -1L; | ||||||||||||||||
| public static final long UNKNOWN_VOLUME_BYTES = -1L; | ||||||||||||||||
|
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. Do we still want this?
Member
Author
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. The wire protocol does not have nullable integers. So the new fields are defined to use -1 as the default value in the protocol: kafka/clients/src/main/resources/common/message/DescribeLogDirsResponse.json Lines 52 to 56 in a73b96a
This constant is used to map the value in DescribeLogDirsResponse to an Optional: kafka/clients/src/main/java/org/apache/kafka/clients/admin/LogDirDescription.java Lines 44 to 45 in a73b96a
|
||||||||||||||||
|
|
||||||||||||||||
| private final DescribeLogDirsResponseData data; | ||||||||||||||||
|
|
||||||||||||||||
|
|
||||||||||||||||
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.
Instead of UNKNOWN_VOLUME_BYTES, this could be
nullor Optional.Empty()? That way we could get rid ofUNKNOWN_VOLUME_BYTEScompletely.