Skip to content

KAFKA-13958: Expose logdirs total and usable space via Kafka API (KIP…#12248

Merged
mimaison merged 5 commits intoapache:trunkfrom
mimaison:kip-827
Jun 14, 2022
Merged

KAFKA-13958: Expose logdirs total and usable space via Kafka API (KIP…#12248
mimaison merged 5 commits intoapache:trunkfrom
mimaison:kip-827

Conversation

@mimaison
Copy link
Member

@mimaison mimaison commented Jun 3, 2022

…-827)

This implements KIP-827: https://cwiki.apache.org/confluence/display/KAFKA/KIP-827%3A+Expose+logdirs+total+and+usable+space+via+Kafka+API

Add TotalBytes and UsableBytes to DescribeLogDirsResponse
Add matching getters on LogDirDescription

Committer Checklist (excluded from commit message)

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

Copy link
Member

@soarez soarez left a comment

Choose a reason for hiding this comment

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

Thanks for this PR

mimaison added 2 commits June 7, 2022 17:02
…-827)

This implements KIP-827: https://cwiki.apache.org/confluence/display/KAFKA/KIP-827%3A+Expose+logdirs+total+and+usable+space+via+Kafka+API

Add TotalBytes and UsableBytes to DescribeLogDirsResponse
Add matching getters on LogDirDescription
assertEquals(mockLogMgr.liveLogDirs.size, responses.size)
responses.foreach { response =>
assertEquals(Errors.NONE.code, response.errorCode)
assertTrue(response.totalBytes >= 0)
Copy link
Member

Choose a reason for hiding this comment

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

Is there a reason due to which totalBytes could be 0? Perhaps we want to assert a > 0 condition here?

Copy link
Member Author

Choose a reason for hiding this comment

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

True this assertion is a bit too defensive! > 0 should be fine. I'll push an update.

public class DescribeLogDirsResponse extends AbstractResponse {

public static final long INVALID_OFFSET_LAG = -1L;
public static final long UNKNOWN_VOLUME_BYTES = -1L;
Copy link
Member

Choose a reason for hiding this comment

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

Do we still want this?
Correct me if I am wrong here but I thought that we reached a conclusion in the KIP that Optional will cover the scenario when client is using a newer API and broker is old.

Copy link
Member Author

Choose a reason for hiding this comment

The 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:

{ "name": "TotalBytes", "type": "int64", "versions": "4+", "ignorable": true, "default": "-1",
"about": "The total size in bytes of the volume the log directory is in."
},
{ "name": "UsableBytes", "type": "int64", "versions": "4+", "ignorable": true, "default": "-1",
"about": "The usable size in bytes of the volume the log directory is in."

This constant is used to map the value in DescribeLogDirsResponse to an Optional:

this.totalBytes = (totalBytes == UNKNOWN_VOLUME_BYTES) ? OptionalLong.empty() : OptionalLong.of(totalBytes);
this.usableBytes = (usableBytes == UNKNOWN_VOLUME_BYTES) ? OptionalLong.empty() : OptionalLong.of(usableBytes);

private final OptionalLong usableBytes;

public LogDirDescription(ApiException error, Map<TopicPartition, ReplicaInfo> replicaInfos) {
this(error, replicaInfos, UNKNOWN_VOLUME_BYTES, UNKNOWN_VOLUME_BYTES);
Copy link
Member

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 null or Optional.Empty()? That way we could get rid of UNKNOWN_VOLUME_BYTES completely.

@mimaison mimaison added the kip Requires or implements a KIP label Jun 8, 2022
Copy link
Member

@tombentley tombentley left a comment

Choose a reason for hiding this comment

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

Thanks @mimaison

}
}
result.put(logDirResult.logDir(), new LogDirDescription(Errors.forCode(logDirResult.errorCode()).exception(), replicaInfoMap));
result.put(logDirResult.logDir(), new LogDirDescription(Errors.forCode(logDirResult.errorCode()).exception(), replicaInfoMap, logDirResult.totalBytes(), logDirResult.usableBytes()));
Copy link
Member

Choose a reason for hiding this comment

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

long line

return totalBytes;
}

public OptionalLong usableBytes() {
Copy link
Member

Choose a reason for hiding this comment

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

Javadoc

Copy link
Member

Choose a reason for hiding this comment

The 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).

Copy link
Member Author

Choose a reason for hiding this comment

The 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

@divijvaidya
Copy link
Member

divijvaidya commented Jun 8, 2022

Thank you for making the suggested changes.

The current set of changes don't contain a fully end to end integration test (KafkaAdminClientTest uses response mocks). Please make a change in DescribeLogDirsRequestTest.scala to validate the new changes.

@mimaison
Copy link
Member Author

Thanks @divijvaidya for the review. I've updated the end to end tests to cover this new feature.

Copy link
Member

@divijvaidya divijvaidya left a comment

Choose a reason for hiding this comment

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

LGTM!

Copy link
Contributor

@viktorsomogyi viktorsomogyi left a comment

Choose a reason for hiding this comment

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

@mimaison thanks for this feature, it's a really useful addition for us as well.
Tried out your code locally, ran the tests, they all work. I don't have any particular comments, it looks good to me.

Copy link
Member

@tombentley tombentley left a comment

Choose a reason for hiding this comment

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

LGTM, thanks @mimaison!

@mimaison mimaison merged commit 4fcfd9d into apache:trunk Jun 14, 2022
@mimaison mimaison deleted the kip-827 branch June 14, 2022 12:20
Sign up for free to join this conversation on GitHub. Already have an account? Sign in to comment

Labels

kip Requires or implements a KIP

Projects

None yet

Development

Successfully merging this pull request may close these issues.

5 participants