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/05/19 17:27:12 UTC

[GitHub] [kafka] jsancio opened a new pull request, #12183: KAFKA-13883: Implement NoOpRecord and metadata metrics

jsancio opened a new pull request, #12183:
URL: https://github.com/apache/kafka/pull/12183

   
   
   ### Committer Checklist (excluded from commit message)
   - [ ] Verify design and implementation 
   - [ ] Verify test coverage and CI build status
   - [ ] Verify documentation (including upgrade notes)
   


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


[GitHub] [kafka] jsancio commented on a diff in pull request #12183: KAFKA-13883: Implement NoOpRecord and metadata metrics

Posted by GitBox <gi...@apache.org>.
jsancio commented on code in PR #12183:
URL: https://github.com/apache/kafka/pull/12183#discussion_r882940851


##########
core/src/main/scala/kafka/server/ControllerServer.scala:
##########
@@ -175,6 +174,11 @@ class ControllerServer(
           OptionalLong.empty()
         }
 
+        val maxIdleIntervalNs = config.metadataMaxIdleIntervalMs match {

Review Comment:
   Okay. I fixed this. `asJava` converts it to a `Optional[Long]` so I did it manually.



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


[GitHub] [kafka] jsancio commented on a diff in pull request #12183: KAFKA-13883: Implement NoOpRecord and metadata metrics

Posted by GitBox <gi...@apache.org>.
jsancio commented on code in PR #12183:
URL: https://github.com/apache/kafka/pull/12183#discussion_r882941312


##########
core/src/main/scala/kafka/server/KafkaConfig.scala:
##########
@@ -1661,6 +1666,10 @@ class KafkaConfig private(doLog: Boolean, val props: java.util.Map[_, _], dynami
 
   /************* Metadata Configuration ***********/
   val metadataSnapshotMaxNewRecordBytes = getLong(KafkaConfig.MetadataSnapshotMaxNewRecordBytesProp)
+  val metadataMaxIdleIntervalMs: Option[Int] = {

Review Comment:
   Done.



##########
core/src/main/scala/kafka/server/KafkaConfig.scala:
##########
@@ -1148,6 +1153,7 @@ object KafkaConfig {
       .define(MetadataLogSegmentMillisProp, LONG, Defaults.LogRollHours * 60 * 60 * 1000L, null, HIGH, MetadataLogSegmentMillisDoc)
       .define(MetadataMaxRetentionBytesProp, LONG, Defaults.LogRetentionBytes, null, HIGH, MetadataMaxRetentionBytesDoc)
       .define(MetadataMaxRetentionMillisProp, LONG, Defaults.LogRetentionHours * 60 * 60 * 1000L, null, HIGH, MetadataMaxRetentionMillisDoc)
+      .define(MetadataMaxIdleIntervalMsProp, INT, Defaults.MetadataMaxIdleIntervalMs, null, LOW, MetadataMaxIdleIntervalMsDoc)

Review Comment:
   Done.



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


[GitHub] [kafka] cmccabe commented on a diff in pull request #12183: KAFKA-13883: Implement NoOpRecord and metadata metrics

Posted by GitBox <gi...@apache.org>.
cmccabe commented on code in PR #12183:
URL: https://github.com/apache/kafka/pull/12183#discussion_r882188489


##########
core/src/main/scala/kafka/server/KafkaConfig.scala:
##########
@@ -1148,6 +1153,7 @@ object KafkaConfig {
       .define(MetadataLogSegmentMillisProp, LONG, Defaults.LogRollHours * 60 * 60 * 1000L, null, HIGH, MetadataLogSegmentMillisDoc)
       .define(MetadataMaxRetentionBytesProp, LONG, Defaults.LogRetentionBytes, null, HIGH, MetadataMaxRetentionBytesDoc)
       .define(MetadataMaxRetentionMillisProp, LONG, Defaults.LogRetentionHours * 60 * 60 * 1000L, null, HIGH, MetadataMaxRetentionMillisDoc)
+      .define(MetadataMaxIdleIntervalMsProp, INT, Defaults.MetadataMaxIdleIntervalMs, null, LOW, MetadataMaxIdleIntervalMsDoc)

Review Comment:
   can we valiate that this is not negative? like I wrote above, there seems to be no reason to allow negative values



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


[GitHub] [kafka] jsancio commented on pull request #12183: KAFKA-13883: Implement NoOpRecord and metadata metrics

Posted by GitBox <gi...@apache.org>.
jsancio commented on PR #12183:
URL: https://github.com/apache/kafka/pull/12183#issuecomment-1143948780

   Merging since the failure are unrelated. The failure at in the ZK tests and this is a KRaft only change:
   ```
   testTopicIdUpgradeAfterReassigningPartitions() – kafka.controller.ControllerIntegrationTest
   ```


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


[GitHub] [kafka] cmccabe merged pull request #12183: KAFKA-13883: Implement NoOpRecord and metadata metrics

Posted by GitBox <gi...@apache.org>.
cmccabe merged PR #12183:
URL: https://github.com/apache/kafka/pull/12183


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


[GitHub] [kafka] cmccabe commented on a diff in pull request #12183: KAFKA-13883: Implement NoOpRecord and metadata metrics

Posted by GitBox <gi...@apache.org>.
cmccabe commented on code in PR #12183:
URL: https://github.com/apache/kafka/pull/12183#discussion_r882192056


##########
metadata/src/main/java/org/apache/kafka/controller/QuorumController.java:
##########
@@ -302,12 +302,30 @@ public QuorumController build() throws Exception {
             KafkaEventQueue queue = null;
             try {
                 queue = new KafkaEventQueue(time, logContext, threadNamePrefix + "QuorumController");
-                return new QuorumController(logContext, nodeId, clusterId, queue, time,
-                    configSchema, raftClient, quorumFeatures, defaultReplicationFactor,
-                    defaultNumPartitions, isLeaderRecoverySupported, replicaPlacer, snapshotMaxNewRecordBytes,
-                    leaderImbalanceCheckIntervalNs, sessionTimeoutNs, controllerMetrics,
-                    createTopicPolicy, alterConfigPolicy, configurationValidator, authorizer,
-                    staticConfig, bootstrapMetadata);
+                return new QuorumController(

Review Comment:
   sigh. all right, fine. I guess we might as well do one line per parameter.



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


[GitHub] [kafka] cmccabe commented on a diff in pull request #12183: KAFKA-13883: Implement NoOpRecord and metadata metrics

Posted by GitBox <gi...@apache.org>.
cmccabe commented on code in PR #12183:
URL: https://github.com/apache/kafka/pull/12183#discussion_r882187967


##########
core/src/main/scala/kafka/server/KafkaConfig.scala:
##########
@@ -712,6 +714,9 @@ object KafkaConfig {
   val MetadataLogDirDoc = "This configuration determines where we put the metadata log for clusters in KRaft mode. " +
     "If it is not set, the metadata log is placed in the first log directory from log.dirs."
   val MetadataSnapshotMaxNewRecordBytesDoc = "This is the maximum number of bytes in the log between the latest snapshot and the high-watermark needed before generating a new snapshot."
+  val MetadataMaxIdleIntervalMsDoc = "This configuration controls how often the active " +

Review Comment:
   Can we disallow negative values for this? 0 for disabled seems clear enough.



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


[GitHub] [kafka] cmccabe commented on a diff in pull request #12183: KAFKA-13883: Implement NoOpRecord and metadata metrics

Posted by GitBox <gi...@apache.org>.
cmccabe commented on code in PR #12183:
URL: https://github.com/apache/kafka/pull/12183#discussion_r882191795


##########
metadata/src/main/java/org/apache/kafka/controller/QuorumController.java:
##########
@@ -155,10 +156,10 @@ static public class Builder {
         private QuorumFeatures quorumFeatures = null;
         private short defaultReplicationFactor = 3;
         private int defaultNumPartitions = 1;
-        private boolean isLeaderRecoverySupported = false;
         private ReplicaPlacer replicaPlacer = new StripedReplicaPlacer(new Random());
         private long snapshotMaxNewRecordBytes = Long.MAX_VALUE;
         private OptionalLong leaderImbalanceCheckIntervalNs = OptionalLong.empty();
+        private OptionalLong maxIdleIntervalNs = OptionalLong.empty();

Review Comment:
   It's an interesting choice to have this off by default. I guess it probably would complicate a bunch of junit tests since it would create records we're not expecting. So maybe this is the correct way to go for now.



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


[GitHub] [kafka] cmccabe commented on a diff in pull request #12183: KAFKA-13883: Implement NoOpRecord and metadata metrics

Posted by GitBox <gi...@apache.org>.
cmccabe commented on code in PR #12183:
URL: https://github.com/apache/kafka/pull/12183#discussion_r882191187


##########
metadata/src/main/java/org/apache/kafka/controller/ReplicationControlManager.java:
##########
@@ -200,23 +201,24 @@ Builder setCreateTopicPolicy(Optional<CreateTopicPolicy> createTopicPolicy) {
         }
 
         ReplicationControlManager build() {
-            if (configurationControl == null) {
-                throw new RuntimeException("You must specify configurationControl.");
-            }
-            if (clusterControl == null) {
-                throw new RuntimeException("You must specify clusterControl.");
-            }
-            if (controllerMetrics == null) {
-                throw new RuntimeException("You must specify controllerMetrics.");
+            if (metadataVersion == null) {
+                throw new IllegalStateException("Metadata version must be set before building");

Review Comment:
   Let's just default to the latest version. It's very tedious to have to set this in every single unit test (most of which don't care)



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


[GitHub] [kafka] jsancio commented on a diff in pull request #12183: KAFKA-13883: Implement NoOpRecord and metadata metrics

Posted by GitBox <gi...@apache.org>.
jsancio commented on code in PR #12183:
URL: https://github.com/apache/kafka/pull/12183#discussion_r882941542


##########
core/src/main/scala/kafka/server/metadata/BrokerMetrics.scala:
##########
@@ -0,0 +1,88 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package kafka.server.metadata
+
+import java.util.concurrent.atomic.AtomicLong
+import org.apache.kafka.common.MetricName
+import org.apache.kafka.common.metrics.Gauge
+import org.apache.kafka.common.metrics.Metrics
+import org.apache.kafka.common.metrics.MetricConfig
+
+final class BrokerMetrics private (metrics: Metrics) extends AutoCloseable {

Review Comment:
   Sounds good to me.



##########
metadata/src/test/java/org/apache/kafka/controller/QuorumControllerTestEnv.java:
##########
@@ -18,10 +18,15 @@
 package org.apache.kafka.controller;
 
 import static java.util.concurrent.TimeUnit.NANOSECONDS;
-
+import java.util.ArrayList;

Review Comment:
   Done.



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


[GitHub] [kafka] jsancio commented on a diff in pull request #12183: KAFKA-13883: Implement NoOpRecord and metadata metrics

Posted by GitBox <gi...@apache.org>.
jsancio commented on code in PR #12183:
URL: https://github.com/apache/kafka/pull/12183#discussion_r882941755


##########
metadata/src/main/java/org/apache/kafka/controller/ReplicationControlManager.java:
##########
@@ -200,23 +201,24 @@ Builder setCreateTopicPolicy(Optional<CreateTopicPolicy> createTopicPolicy) {
         }
 
         ReplicationControlManager build() {
-            if (configurationControl == null) {
-                throw new RuntimeException("You must specify configurationControl.");
-            }
-            if (clusterControl == null) {
-                throw new RuntimeException("You must specify clusterControl.");
-            }
-            if (controllerMetrics == null) {
-                throw new RuntimeException("You must specify controllerMetrics.");
+            if (metadataVersion == null) {
+                throw new IllegalStateException("Metadata version must be set before building");

Review Comment:
   Sounds good to me.



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


[GitHub] [kafka] cmccabe commented on a diff in pull request #12183: KAFKA-13883: Implement NoOpRecord and metadata metrics

Posted by GitBox <gi...@apache.org>.
cmccabe commented on code in PR #12183:
URL: https://github.com/apache/kafka/pull/12183#discussion_r882190022


##########
server-common/src/main/java/org/apache/kafka/server/common/MetadataVersion.java:
##########
@@ -150,10 +150,13 @@ public enum MetadataVersion {
     IBP_3_1_IV0(3, "3.1", "IV0", false),
 
     // Support for leader recovery for unclean leader election (KIP-704)
-    IBP_3_2_IV0(4, "3.2", "IV0", false),
+    IBP_3_2_IV0(4, "3.2", "IV0", true),

Review Comment:
   Thanks for fixing this. We'll have to make a note in the PR description as well



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


[GitHub] [kafka] jsancio commented on a diff in pull request #12183: KAFKA-13883: Implement NoOpRecord and metadata metrics

Posted by GitBox <gi...@apache.org>.
jsancio commented on code in PR #12183:
URL: https://github.com/apache/kafka/pull/12183#discussion_r882941106


##########
core/src/main/scala/kafka/server/KafkaConfig.scala:
##########
@@ -712,6 +714,9 @@ object KafkaConfig {
   val MetadataLogDirDoc = "This configuration determines where we put the metadata log for clusters in KRaft mode. " +
     "If it is not set, the metadata log is placed in the first log directory from log.dirs."
   val MetadataSnapshotMaxNewRecordBytesDoc = "This is the maximum number of bytes in the log between the latest snapshot and the high-watermark needed before generating a new snapshot."
+  val MetadataMaxIdleIntervalMsDoc = "This configuration controls how often the active " +

Review Comment:
   Done.



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


[GitHub] [kafka] cmccabe commented on a diff in pull request #12183: KAFKA-13883: Implement NoOpRecord and metadata metrics

Posted by GitBox <gi...@apache.org>.
cmccabe commented on code in PR #12183:
URL: https://github.com/apache/kafka/pull/12183#discussion_r882186601


##########
core/src/main/scala/kafka/server/ControllerServer.scala:
##########
@@ -175,6 +174,11 @@ class ControllerServer(
           OptionalLong.empty()
         }
 
+        val maxIdleIntervalNs = config.metadataMaxIdleIntervalMs match {

Review Comment:
   It's more traditional to have conversion stuff like this in KafkaConfig. If you had a function like `maxIdleIntervalNs` there, it could return Option[Long] and we could convert that into OptionalLong as needed (I think a simple asScala can do it if you have the optional converter imported...)



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


[GitHub] [kafka] cmccabe commented on a diff in pull request #12183: KAFKA-13883: Implement NoOpRecord and metadata metrics

Posted by GitBox <gi...@apache.org>.
cmccabe commented on code in PR #12183:
URL: https://github.com/apache/kafka/pull/12183#discussion_r882194131


##########
metadata/src/main/java/org/apache/kafka/controller/QuorumController.java:
##########
@@ -1010,6 +1028,38 @@ private void maybeCompleteAuthorizerInitialLoad() {
         }
     }
 
+    private boolean isActiveController() {
+        return curClaimEpoch != -1;
+    }
+
+    private void updateWriteOffset(long offset) {
+        writeOffset = offset;
+        if (isActiveController()) {
+            controllerMetrics.setLastAppliedRecordOffset(writeOffset);
+            // This is not truly the append timestamp. The KRaft client doesn't expose the append time when scheduling a write.
+            // This is good enough because this is called right after the records were given to the KRAft client for appending and
+            // the default append linger for KRaft is 25ms.
+            controllerMetrics.setLastAppliedRecordTimestamp(time.milliseconds());
+        } else {
+            // This is called with an offset of -1 when the active controller renounced leadership

Review Comment:
   Maybe the comment should just mention that lastAppliedRecordOffset tracks last committed offset for standbys.
   
   The bit about writeOffset being -1 is a bit low-level (it is true but that feels more like an implementation detail here...)



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


[GitHub] [kafka] jsancio commented on a diff in pull request #12183: KAFKA-13883: Implement NoOpRecord and metadata metrics

Posted by GitBox <gi...@apache.org>.
jsancio commented on code in PR #12183:
URL: https://github.com/apache/kafka/pull/12183#discussion_r882942653


##########
metadata/src/main/java/org/apache/kafka/controller/QuorumController.java:
##########
@@ -302,12 +302,30 @@ public QuorumController build() throws Exception {
             KafkaEventQueue queue = null;
             try {
                 queue = new KafkaEventQueue(time, logContext, threadNamePrefix + "QuorumController");
-                return new QuorumController(logContext, nodeId, clusterId, queue, time,
-                    configSchema, raftClient, quorumFeatures, defaultReplicationFactor,
-                    defaultNumPartitions, isLeaderRecoverySupported, replicaPlacer, snapshotMaxNewRecordBytes,
-                    leaderImbalanceCheckIntervalNs, sessionTimeoutNs, controllerMetrics,
-                    createTopicPolicy, alterConfigPolicy, configurationValidator, authorizer,
-                    staticConfig, bootstrapMetadata);
+                return new QuorumController(

Review Comment:
   Hehe. I thought it was improvement. I always find the previous style harder to read and update.



##########
metadata/src/main/java/org/apache/kafka/controller/QuorumController.java:
##########
@@ -1010,6 +1028,38 @@ private void maybeCompleteAuthorizerInitialLoad() {
         }
     }
 
+    private boolean isActiveController() {
+        return curClaimEpoch != -1;
+    }
+
+    private void updateWriteOffset(long offset) {
+        writeOffset = offset;
+        if (isActiveController()) {
+            controllerMetrics.setLastAppliedRecordOffset(writeOffset);
+            // This is not truly the append timestamp. The KRaft client doesn't expose the append time when scheduling a write.
+            // This is good enough because this is called right after the records were given to the KRAft client for appending and
+            // the default append linger for KRaft is 25ms.
+            controllerMetrics.setLastAppliedRecordTimestamp(time.milliseconds());
+        } else {
+            // This is called with an offset of -1 when the active controller renounced leadership

Review Comment:
   Done.



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


[GitHub] [kafka] cmccabe commented on a diff in pull request #12183: KAFKA-13883: Implement NoOpRecord and metadata metrics

Posted by GitBox <gi...@apache.org>.
cmccabe commented on code in PR #12183:
URL: https://github.com/apache/kafka/pull/12183#discussion_r882189435


##########
core/src/main/scala/kafka/server/metadata/BrokerMetrics.scala:
##########
@@ -0,0 +1,88 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package kafka.server.metadata
+
+import java.util.concurrent.atomic.AtomicLong
+import org.apache.kafka.common.MetricName
+import org.apache.kafka.common.metrics.Gauge
+import org.apache.kafka.common.metrics.Metrics
+import org.apache.kafka.common.metrics.MetricConfig
+
+final class BrokerMetrics private (metrics: Metrics) extends AutoCloseable {

Review Comment:
   How do you feel about `BrokerServerMetrics` as a name for this? That might make it more clear that this doesn't apply to `KafkaServer`...



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


[GitHub] [kafka] cmccabe commented on a diff in pull request #12183: KAFKA-13883: Implement NoOpRecord and metadata metrics

Posted by GitBox <gi...@apache.org>.
cmccabe commented on code in PR #12183:
URL: https://github.com/apache/kafka/pull/12183#discussion_r882188358


##########
core/src/main/scala/kafka/server/KafkaConfig.scala:
##########
@@ -1661,6 +1666,10 @@ class KafkaConfig private(doLog: Boolean, val props: java.util.Map[_, _], dynami
 
   /************* Metadata Configuration ***********/
   val metadataSnapshotMaxNewRecordBytes = getLong(KafkaConfig.MetadataSnapshotMaxNewRecordBytesProp)
+  val metadataMaxIdleIntervalMs: Option[Int] = {

Review Comment:
   Like I wrote above, seems like the accessor we want should give us nanoseconds not milliseconds. I suppose the value in nanoseconds should be a long too...



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


[GitHub] [kafka] cmccabe commented on a diff in pull request #12183: KAFKA-13883: Implement NoOpRecord and metadata metrics

Posted by GitBox <gi...@apache.org>.
cmccabe commented on code in PR #12183:
URL: https://github.com/apache/kafka/pull/12183#discussion_r882190505


##########
metadata/src/test/java/org/apache/kafka/controller/QuorumControllerTestEnv.java:
##########
@@ -18,10 +18,15 @@
 package org.apache.kafka.controller;
 
 import static java.util.concurrent.TimeUnit.NANOSECONDS;
-
+import java.util.ArrayList;

Review Comment:
   I kind of prefer the style where all the java.util imports are in a single block, can we do that here?



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