You are viewing a plain text version of this content. The canonical link for it is here.
Posted to jira@kafka.apache.org by GitBox <gi...@apache.org> on 2022/06/08 11:18:51 UTC

[GitHub] [kafka] divijvaidya commented on a diff in pull request #12248: KAFKA-13958: Expose logdirs total and usable space via Kafka API (KIP…

divijvaidya commented on code in PR #12248:
URL: https://github.com/apache/kafka/pull/12248#discussion_r892224792


##########
core/src/test/scala/unit/kafka/server/ReplicaManagerTest.scala:
##########
@@ -4144,6 +4148,31 @@ class ReplicaManagerTest {
       replicaManager.shutdown(checkpointHW = false)
     }
   }
+
+  @Test
+  def testDescribeLogDirs(): Unit = {
+    val topicPartition = 0
+    val topicId = Uuid.randomUuid()
+    val followerBrokerId = 0
+    val leaderBrokerId = 1
+    val leaderEpoch = 1
+    val leaderEpochIncrement = 2
+    val countDownLatch = new CountDownLatch(1)
+    val offsetFromLeader = 5
+
+    // Prepare the mocked components for the test
+    val (replicaManager, mockLogMgr) = prepareReplicaManagerAndLogManager(new MockTimer(time),
+      topicPartition, leaderEpoch + leaderEpochIncrement, followerBrokerId, leaderBrokerId, countDownLatch,
+      expectTruncation = false, localLogOffset = Some(10), offsetFromLeader = offsetFromLeader, topicId = Some(topicId))
+
+    val responses = replicaManager.describeLogDirs(Set(new TopicPartition(topic, topicPartition)))
+    assertEquals(mockLogMgr.liveLogDirs.size, responses.size)
+    responses.foreach { response =>
+      assertEquals(Errors.NONE.code, response.errorCode)
+      assertTrue(response.totalBytes >= 0)

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



##########
clients/src/main/java/org/apache/kafka/common/requests/DescribeLogDirsResponse.java:
##########
@@ -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;

Review Comment:
   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.



##########
clients/src/main/java/org/apache/kafka/clients/admin/LogDirDescription.java:
##########
@@ -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);

Review Comment:
   Instead of UNKNOWN_VOLUME_BYTES, this could be `null` or Optional.Empty()? That way we could get rid of `UNKNOWN_VOLUME_BYTES` completely.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: jira-unsubscribe@kafka.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org